diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index 47b1239f..fc3400e8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -530,7 +530,8 @@ class ConsumerStates { consumer.isIgnoreUnexpectedChildShards(), consumer.getLeaseCoordinator(), consumer.getTaskBackoffTimeMillis(), - consumer.getGetRecordsCache(), consumer.getShardSyncer(), consumer.getShardSyncStrategy()); + consumer.getGetRecordsCache(), consumer.getShardSyncer(), + consumer.getShardSyncStrategy(), consumer.getChildShards()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java index e425e070..c716afa1 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java @@ -16,7 +16,12 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.Collections; import java.util.Date; +import java.util.List; +import java.util.Set; +import com.amazonaws.SdkClientException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.model.ChildShard; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -47,6 +52,7 @@ class KinesisDataFetcher { private boolean isInitialized; private String lastKnownSequenceNumber; private InitialPositionInStreamExtended initialPositionInStream; + private List childShards = Collections.emptyList(); /** * @@ -85,8 +91,11 @@ class KinesisDataFetcher { final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() { @Override public GetRecordsResult getResult() { - return new GetRecordsResult().withMillisBehindLatest(null).withRecords(Collections.emptyList()) - .withNextShardIterator(null); + return new GetRecordsResult() + .withMillisBehindLatest(null) + .withRecords(Collections.emptyList()) + .withNextShardIterator(null) + .withChildShards(Collections.emptyList()); } @Override @@ -113,12 +122,20 @@ class KinesisDataFetcher { @Override public GetRecordsResult accept() { + if (!isValidResult(result)) { + // Throwing SDK exception when the GetRecords result is not valid. This will allow PrefetchGetRecordsCache to retry the GetRecords call. + throw new SdkClientException("Shard " + shardId +": GetRecordsResult is not valid. NextShardIterator: " + result.getNextShardIterator() + + ". ChildShards: " + result.getChildShards()); + } nextIterator = result.getNextShardIterator(); if (!CollectionUtils.isNullOrEmpty(result.getRecords())) { lastKnownSequenceNumber = Iterables.getLast(result.getRecords()).getSequenceNumber(); } if (nextIterator == null) { - LOG.info("Reached shard end: nextIterator is null in AdvancingResult.accept for shard " + shardId); + LOG.info("Reached shard end: nextIterator is null in AdvancingResult.accept for shard " + shardId + ". childShards: " + result.getChildShards()); + if (!CollectionUtils.isNullOrEmpty(result.getChildShards())) { + childShards = result.getChildShards(); + } isShardEndReached = true; } return getResult(); @@ -130,6 +147,23 @@ class KinesisDataFetcher { } } + private boolean isValidResult(GetRecordsResult getRecordsResult) { + // GetRecords result should contain childShard information. There are two valid combination for the nextShardIterator and childShards + // If the GetRecords call does not reach the shard end, getRecords result should contain a non-null nextShardIterator and an empty list of childShards. + // If the GetRecords call reaches the shard end, getRecords result should contain a null nextShardIterator and a non-empty list of childShards. + // All other combinations are invalid and indicating an issue with GetRecords result from Kinesis service. + if (getRecordsResult.getNextShardIterator() == null && CollectionUtils.isNullOrEmpty(getRecordsResult.getChildShards()) || + getRecordsResult.getNextShardIterator() != null && !CollectionUtils.isNullOrEmpty(getRecordsResult.getChildShards())) { + return false; + } + for (ChildShard childShard : getRecordsResult.getChildShards()) { + if (CollectionUtils.isNullOrEmpty(childShard.getParentShards())) { + return false; + } + } + return true; + } + /** * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number. * @param initialCheckpoint Current checkpoint sequence number for this shard. @@ -141,8 +175,7 @@ class KinesisDataFetcher { isInitialized = true; } - public void initialize(ExtendedSequenceNumber initialCheckpoint, - InitialPositionInStreamExtended initialPositionInStream) { + public void initialize(ExtendedSequenceNumber initialCheckpoint, InitialPositionInStreamExtended initialPositionInStream) { LOG.info("Initializing shard " + shardId + " with " + initialCheckpoint.getSequenceNumber()); advanceIteratorTo(initialCheckpoint.getSequenceNumber(), initialPositionInStream); isInitialized = true; @@ -171,6 +204,7 @@ class KinesisDataFetcher { if (nextIterator == null) { LOG.info("Reached shard end: cannot advance iterator for shard " + shardId); isShardEndReached = true; + // TODO: transition to ShuttingDown state on shardend instead to shutdown state for enqueueing this for cleanup } this.lastKnownSequenceNumber = sequenceNumber; this.initialPositionInStream = initialPositionInStream; @@ -248,6 +282,10 @@ class KinesisDataFetcher { return isShardEndReached; } + protected List getChildShards() { + return childShards; + } + /** Note: This method has package level access for testing purposes. * @return nextIterator */ diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index 21890663..08543230 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -28,9 +28,11 @@ import java.util.Set; import com.amazonaws.services.kinesis.leases.impl.Lease; import com.amazonaws.services.kinesis.leases.impl.LeaseManager; +import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; import com.amazonaws.util.CollectionUtils; +import lombok.NoArgsConstructor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.lang3.StringUtils; @@ -457,7 +459,7 @@ class KinesisShardSyncer implements ShardSyncer { /** * Note: Package level access for testing purposes only. * Check if this shard is a descendant of a shard that is (or will be) processed. - * Create leases for the ancestors of this shard as required. + * Create leases for the first ancestor of this shard that needs to be processed, as required. * See javadoc of determineNewLeasesToCreate() for rules and example. * * @param shardId The shardId to check. @@ -473,9 +475,10 @@ class KinesisShardSyncer implements ShardSyncer { static boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId, InitialPositionInStreamExtended initialPosition, Set shardIdsOfCurrentLeases, Map shardIdToShardMapOfAllKinesisShards, - Map shardIdToLeaseMapOfNewShards, Map memoizationContext) { + Map shardIdToLeaseMapOfNewShards, MemoizationContext memoizationContext) { + + final Boolean previousValue = memoizationContext.isDescendant(shardId); - Boolean previousValue = memoizationContext.get(shardId); if (previousValue != null) { return previousValue; } @@ -495,10 +498,13 @@ class KinesisShardSyncer implements ShardSyncer { shard = shardIdToShardMapOfAllKinesisShards.get(shardId); parentShardIds = getParentShardIds(shard, shardIdToShardMapOfAllKinesisShards); for (String parentShardId : parentShardIds) { - // Check if the parent is a descendant, and include its ancestors. - if (checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, initialPosition, - shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, - memoizationContext)) { + // Check if the parent is a descendant, and include its ancestors. Or, if the parent is NOT a + // descendant but we should create a lease for it anyway (e.g. to include in processing from + // TRIM_HORIZON or AT_TIMESTAMP). If either is true, then we mark the current shard as a descendant. + final boolean isParentDescendant = checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, + initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, + shardIdToLeaseMapOfNewShards, memoizationContext); + if (isParentDescendant || memoizationContext.shouldCreateLease(parentShardId)) { isDescendant = true; descendantParentShardIds.add(parentShardId); LOG.debug("Parent shard " + parentShardId + " is a descendant."); @@ -511,37 +517,76 @@ class KinesisShardSyncer implements ShardSyncer { if (isDescendant) { for (String parentShardId : parentShardIds) { if (!shardIdsOfCurrentLeases.contains(parentShardId)) { - LOG.debug("Need to create a lease for shardId " + parentShardId); KinesisClientLease lease = shardIdToLeaseMapOfNewShards.get(parentShardId); + + // If the lease for the parent shard does not already exist, there are two cases in which we + // would want to create it: + // - If we have already marked the parentShardId for lease creation in a prior recursive + // call. This could happen if we are trying to process from TRIM_HORIZON or AT_TIMESTAMP. + // - If the parent shard is not a descendant but the current shard is a descendant, then + // the parent shard is the oldest shard in the shard hierarchy that does not have an + // ancestor in the lease table (the adjacent parent is necessarily a descendant, and + // therefore covered in the lease table). So we should create a lease for the parent. + if (lease == null) { - lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); - shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + if (memoizationContext.shouldCreateLease(parentShardId) || + !descendantParentShardIds.contains(parentShardId)) { + LOG.debug("Need to create a lease for shardId " + parentShardId); + lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); + shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + } } - if (descendantParentShardIds.contains(parentShardId) && !initialPosition - .getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { - lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - lease.setCheckpoint(convertToCheckpoint(initialPosition)); + /** + * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the + * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will + * add a lease just like we do for TRIM_HORIZON. However we will only return back records + * with server-side timestamp at or after the specified initial position timestamp. + * + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: (4, 5, 7) + * + * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with + * timestamp value 206. We will then create new leases for all the shards 0 and 1 (with + * checkpoint set AT_TIMESTAMP), even though these ancestor shards have an epoch less than + * 206. However as we begin processing the ancestor shards, their checkpoints would be + * updated to SHARD_END and their leases would then be deleted since they won't have records + * with server-side timestamp at/after 206. And after that we will begin processing the + * descendant shards with epoch at/after 206 and we will return the records that meet the + * timestamp requirement for these shards. + */ + if (lease != null) { + if (descendantParentShardIds.contains(parentShardId) && !initialPosition + .getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } else { + lease.setCheckpoint(convertToCheckpoint(initialPosition)); + } } } } } else { - // This shard should be included, if the customer wants to process all records in the stream or - // if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a lease just like we do - // for TRIM_HORIZON. However we will only return back records with server-side timestamp at or - // after the specified initial position timestamp. + // This shard is not a descendant, but should still be included if the customer wants to process all + // records in the stream or if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a + // lease just like we do for TRIM_HORIZON. However we will only return back records with server-side + // timestamp at or after the specified initial position timestamp. if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON) || initialPosition.getInitialPositionInStream() .equals(InitialPositionInStream.AT_TIMESTAMP)) { - isDescendant = true; + memoizationContext.setShouldCreateLease(shardId, true); } } } } - memoizationContext.put(shardId, isDescendant); + memoizationContext.setIsDescendant(shardId, isDescendant); return isDescendant; } // CHECKSTYLE:ON CyclomaticComplexity @@ -735,6 +780,29 @@ class KinesisShardSyncer implements ShardSyncer { return newLease; } + /** + * Helper method to create a new KinesisClientLease POJO for a ChildShard. + * Note: Package level access only for testing purposes + * + * @param childShard + * @return + */ + static KinesisClientLease newKCLLeaseForChildShard(ChildShard childShard) throws InvalidStateException { + final KinesisClientLease newLease = new KinesisClientLease(); + newLease.setLeaseKey(childShard.getShardId()); + final List parentShardIds = new ArrayList<>(); + if (!CollectionUtils.isNullOrEmpty(childShard.getParentShards())) { + parentShardIds.addAll(childShard.getParentShards()); + } else { + throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.getShardId() + + " because parent shards cannot be found."); + } + newLease.setParentShardIds(parentShardIds); + newLease.setOwnerSwitchesSinceCheckpoint(0L); + newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return newLease; + } + /** * Helper method to construct a shardId->Shard map for the specified list of shards. * @@ -834,4 +902,28 @@ class KinesisShardSyncer implements ShardSyncer { } + /** + * Helper class to pass around state between recursive traversals of shard hierarchy. + */ + @NoArgsConstructor + static class MemoizationContext { + private Map isDescendantMap = new HashMap<>(); + private Map shouldCreateLeaseMap = new HashMap<>(); + + Boolean isDescendant(String shardId) { + return isDescendantMap.get(shardId); + } + + void setIsDescendant(String shardId, Boolean isDescendant) { + isDescendantMap.put(shardId, isDescendant); + } + + Boolean shouldCreateLease(String shardId) { + return shouldCreateLeaseMap.computeIfAbsent(shardId, x -> Boolean.FALSE); + } + + void setShouldCreateLease(String shardId, Boolean shouldCreateLease) { + shouldCreateLeaseMap.put(shardId, shouldCreateLease); + } + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java index 53c42980..2a868d30 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java @@ -47,8 +47,10 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { * Note: Package level access only for testing purposes. * * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, - * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): - * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. + * determine if it is a descendant of any shard which is or will be processed (e.g. for which a lease exists): + * If so, create a lease for the first ancestor that needs to be processed (if needed). We will create leases + * for no more than one level in the ancestry tree. Once we find the first ancestor that needs to be processed, + * we will avoid creating leases for further descendants of that ancestor. * If not, set checkpoint of the shard to the initial position specified by the client. * To check if we need to create leases for ancestors, we use the following rules: * * If we began (or will begin) processing data for a shard, then we must reach end of that shard before @@ -67,10 +69,17 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { * 0 1 2 3 4 5 - shards till epoch 102 * \ / \ / | | * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | / \ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) - * New leases to create: (2, 6, 7, 8, 9, 10) + * \ / | / \ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: (4, 5, 7) + * + * If initial position is LATEST: + * - New leases to create: (6) + * If initial position is TRIM_HORIZON: + * - New leases to create: (0, 1) + * If initial position is AT_TIMESTAMP(epoch=200): + * - New leases to create: (0, 1) * * The leases returned are sorted by the starting sequence number - following the same order * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail @@ -104,7 +113,8 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { } List openShards = KinesisShardSyncer.getOpenShards(shards); - Map memoizationContext = new HashMap<>(); + final KinesisShardSyncer.MemoizationContext memoizationContext = new KinesisShardSyncer.MemoizationContext(); + // Iterate over the open shards and find those that don't have any lease entries. for (Shard shard : openShards) { @@ -115,43 +125,30 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { } else if (inconsistentShardIds.contains(shardId)) { LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease"); } else { - LOG.debug("Need to create a lease for shardId " + shardId); - KinesisClientLease newLease = KinesisShardSyncer.newKCLLease(shard); + LOG.debug("Beginning traversal of ancestry tree for shardId " + shardId); + + // A shard is a descendant if at least one if its ancestors exists in the lease table. + // We will create leases for only one level in the ancestry tree. Once we find the first ancestor + // that needs to be processed in order to complete the hash range, we will not create leases for + // further descendants of that ancestor. boolean isDescendant = KinesisShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, memoizationContext); - /** - * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the - * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a - * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side - * timestamp at or after the specified initial position timestamp. - * - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * - * Current leases: empty set - * - * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with - * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to - * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin - * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases - * would then be deleted since they won't have records with server-side timestamp at/after 206. And - * after that we will begin processing the descendant shards with epoch at/after 206 and we will - * return the records that meet the timestamp requirement for these shards. - */ - if (isDescendant && !initialPosition.getInitialPositionInStream() - .equals(InitialPositionInStream.AT_TIMESTAMP)) { - newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { + // If shard is a descendant, the leases for its ancestors were already created above. Open shards + // that are NOT descendants will not have leases yet, so we create them here. We will not create + // leases for open shards that ARE descendants yet - leases for these shards will be created upon + // SHARD_END of their parents. + if (!isDescendant) { + LOG.debug("ShardId " + shardId + " has no ancestors. Creating a lease."); + final KinesisClientLease newLease = KinesisShardSyncer.newKCLLease(shard); newLease.setCheckpoint(KinesisShardSyncer.convertToCheckpoint(initialPosition)); + LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint()); + shardIdToNewLeaseMap.put(shardId, newLease); + } else { + LOG.debug("ShardId " + shardId + " is a descendant whose ancestors should already have leases. " + + "Not creating a lease."); } - LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint()); - shardIdToNewLeaseMap.put(shardId, newLease); } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java index 8173a479..a4cf74d8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java @@ -129,6 +129,7 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { try { result = getRecordsResultQueue.take().withCacheExitTime(Instant.now()); prefetchCounters.removed(result); + log.info("Shard " + shardId + ": Number of records remaining in queue is " + getRecordsResultQueue.size()); } catch (InterruptedException e) { log.error("Interrupted while getting records from the cache", e); } @@ -177,7 +178,6 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { MetricsHelper.getMetricsScope().addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.Count, MetricsLevel.SUMMARY); - dataFetcher.restartIterator(); } catch (SdkClientException e) { log.error("Exception thrown while fetching records from Kinesis", e); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java index b578fbb0..cd543e23 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java @@ -152,8 +152,8 @@ class ProcessTask implements ITask { try { if (dataFetcher.isShardEndReached()) { - LOG.info("Reached end of shard " + shardInfo.getShardId()); - return new TaskResult(null, true); + LOG.info("Reached end of shard " + shardInfo.getShardId() + ". Found childShards: " + dataFetcher.getChildShards()); + return new TaskResult(null, true, dataFetcher.getChildShards()); } final ProcessRecordsInput processRecordsInput = getRecordsResult(); @@ -353,7 +353,7 @@ class ProcessTask implements ITask { * recordProcessorCheckpointer). */ dataFetcher.advanceIteratorTo(recordProcessorCheckpointer.getLargestPermittedCheckpointValue() - .getSequenceNumber(), streamConfig.getInitialPositionInStream()); + .getSequenceNumber(), streamConfig.getInitialPositionInStream()); // Try a second time - if we fail this time, expose the failure. try { diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index a30412ce..f5513d3e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -15,11 +15,14 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; +import com.amazonaws.services.kinesis.model.ChildShard; +import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -66,6 +69,9 @@ class ShardConsumer { private Future future; private ShardSyncStrategy shardSyncStrategy; + @Getter + private List childShards; + @Getter private final GetRecordsCache getRecordsCache; @@ -321,6 +327,10 @@ class ShardConsumer { TaskResult result = future.get(); if (result.getException() == null) { if (result.isShardEndReached()) { + if (!CollectionUtils.isNullOrEmpty(result.getChildShards())) { + childShards = result.getChildShards(); + LOG.info("Shard " + shardInfo.getShardId() + ": Setting childShards in ShardConsumer: " + childShards); + } return TaskOutcome.END_OF_SHARD; } return TaskOutcome.SUCCESSFUL; @@ -420,6 +430,7 @@ class ShardConsumer { void updateState(TaskOutcome taskOutcome) { if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.TERMINATE); + LOG.info("Shard " + shardInfo.getShardId() + ": Mark for shutdown with reason TERMINATE"); } if (isShutdownRequested() && taskOutcome != TaskOutcome.FAILURE) { currentState = currentState.shutdownTransition(shutdownReason); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index a9ff5080..71cf3b9d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -14,9 +14,11 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; -import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardClosureVerificationResponse; -import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; -import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.model.ChildShard; +import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,6 +32,9 @@ import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.google.common.annotations.VisibleForTesting; import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; /** * Task for invoking the RecordProcessor shutdown() callback. @@ -54,6 +59,7 @@ class ShutdownTask implements ITask { private final GetRecordsCache getRecordsCache; private final ShardSyncer shardSyncer; private final ShardSyncStrategy shardSyncStrategy; + private final List childShards; /** * Constructor. @@ -69,7 +75,8 @@ class ShutdownTask implements ITask { boolean ignoreUnexpectedChildShards, KinesisClientLibLeaseCoordinator leaseCoordinator, long backoffTimeMillis, - GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) { + GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, + ShardSyncStrategy shardSyncStrategy, List childShards) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -83,6 +90,7 @@ class ShutdownTask implements ITask { this.getRecordsCache = getRecordsCache; this.shardSyncer = shardSyncer; this.shardSyncStrategy = shardSyncStrategy; + this.childShards = childShards; } /* @@ -97,29 +105,39 @@ class ShutdownTask implements ITask { boolean applicationException = false; try { + LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: " + + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); ShutdownReason localReason = reason; - List latestShards = null; /* * Revalidate if the current shard is closed before shutting down the shard consumer with reason SHARD_END * If current shard is not closed, shut down the shard consumer with reason LEASE_LOST that allows active * workers to contend for the lease of this shard. */ if(localReason == ShutdownReason.TERMINATE) { - ShardClosureVerificationResponse shardClosureVerificationResponse = kinesisProxy.verifyShardClosure(shardInfo.getShardId()); - if (shardClosureVerificationResponse instanceof ShardListWrappingShardClosureVerificationResponse) { - latestShards = ((ShardListWrappingShardClosureVerificationResponse)shardClosureVerificationResponse).getLatestShards(); - } - - // If shard in context is not closed yet we should shut down the ShardConsumer with Zombie state - // which avoids checkpoint-ing with SHARD_END sequence number. - if(!shardClosureVerificationResponse.isShardClosed()) { + // Create new lease for the child shards if they don't exist. + // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. + // This would happen when KinesisDataFetcher catches ResourceNotFound exception. + // In this case, KinesisDataFetcher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. + // This scenario could happen when customer deletes the stream while leaving the KCL application running. + try { + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateCurrentLeaseWithChildShards(); + } else { + LOG.warn("Shard " + shardInfo.getShardId() + + ": Shutting down consumer with SHARD_END reason without creating leases for child shards."); + } + } catch (InvalidStateException e) { + // If invalidStateException happens, it indicates we are missing childShard related information. + // In this scenario, we should shutdown the shardConsumer with ZOMBIE reason to allow other worker to take the lease and retry getting + // childShard information in the processTask. localReason = ShutdownReason.ZOMBIE; dropLease(); - LOG.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfo.getShardId()); + LOG.warn("Shard " + shardInfo.getShardId() + ": Exception happened while shutting down shardConsumer with TERMINATE reason. " + + "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); } } - // If we reached end of the shard, set sequence number to SHARD_END. if (localReason == ShutdownReason.TERMINATE) { recordProcessorCheckpointer.setSequenceNumberAtShardEnd( @@ -127,8 +145,6 @@ class ShutdownTask implements ITask { recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); } - LOG.debug("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken " - + shardInfo.getConcurrencyToken() + ". Shutdown reason: " + localReason); final ShutdownInput shutdownInput = new ShutdownInput() .withShutdownReason(localReason) .withCheckpointer(recordProcessorCheckpointer); @@ -156,18 +172,6 @@ class ShutdownTask implements ITask { MetricsLevel.SUMMARY); } - if (localReason == ShutdownReason.TERMINATE) { - LOG.debug("Looking for child shards of shard " + shardInfo.getShardId()); - // create leases for the child shards - TaskResult result = shardSyncStrategy.onShardConsumerShutDown(latestShards); - if (result.getException() != null) { - LOG.debug("Exception while trying to sync shards on the shutdown of shard: " + shardInfo - .getShardId()); - throw result.getException(); - } - LOG.debug("Finished checking for child shards of shard " + shardInfo.getShardId()); - } - return new TaskResult(null); } catch (Exception e) { if (applicationException) { @@ -187,6 +191,33 @@ class ShutdownTask implements ITask { return new TaskResult(exception); } + private void createLeasesForChildShardsIfNotExist() throws InvalidStateException, DependencyException, ProvisionedThroughputException { + for (ChildShard childShard : childShards) { + final String leaseKey = childShard.getShardId(); + if (leaseCoordinator.getLeaseManager().getLease(leaseKey) == null) { + final KinesisClientLease leaseToCreate = KinesisShardSyncer.newKCLLeaseForChildShard(childShard); + leaseCoordinator.getLeaseManager().createLeaseIfNotExists(leaseToCreate); + LOG.info("Shard " + shardInfo.getShardId() + " : Created child shard lease: " + leaseToCreate.getLeaseKey()); + } + } + } + + private void updateCurrentLeaseWithChildShards() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + if (currentLease == null) { + throw new InvalidStateException("Failed to retrieve current lease for shard " + shardInfo.getShardId()); + } + final Set childShardIds = childShards.stream().map(ChildShard::getShardId).collect(Collectors.toSet()); + + currentLease.setChildShardIds(childShardIds); + final boolean updateResult = leaseCoordinator.updateLease(currentLease, UUID.fromString(shardInfo.getConcurrencyToken())); + if (!updateResult) { + throw new InvalidStateException("Failed to update parent lease with child shard information for shard " + shardInfo.getShardId()); + } + LOG.info("Shard " + shardInfo.getShardId() + ": Updated current lease with child shard information: " + currentLease.getLeaseKey()); + } + + /* * (non-Javadoc) * @@ -204,6 +235,10 @@ class ShutdownTask implements ITask { private void dropLease() { KinesisClientLease lease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + if (lease == null) { + LOG.warn("Shard " + shardInfo.getShardId() + ": Lease already dropped. Will shutdown the shardConsumer directly."); + return; + } leaseCoordinator.dropLease(lease); LOG.warn("Dropped lease for shutting down ShardConsumer: " + lease.getLeaseKey()); } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java index bc68d292..70109b86 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java @@ -14,6 +14,10 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import com.amazonaws.services.kinesis.model.ChildShard; + +import java.util.List; + /** * Used to capture information from a task that we want to communicate back to the higher layer. * E.g. exception thrown when executing the task, if we reach end of a shard. @@ -26,6 +30,9 @@ class TaskResult { // Any exception caught while executing the task. private Exception exception; + // List of childShards of the current shard. This field is only required for the task result when we reach end of a shard. + private List childShards; + /** * @return the shardEndReached */ @@ -33,6 +40,11 @@ class TaskResult { return shardEndReached; } + /** + * @return the list of childShards. + */ + protected List getChildShards() { return childShards; } + /** * @param shardEndReached the shardEndReached to set */ @@ -40,6 +52,11 @@ class TaskResult { this.shardEndReached = shardEndReached; } + /** + * @param childShards the list of childShards to set + */ + protected void setChildShards(List childShards) { this.childShards = childShards; } + /** * @return the exception */ @@ -70,4 +87,10 @@ class TaskResult { this.shardEndReached = isShardEndReached; } + TaskResult(Exception e, boolean isShardEndReached, List childShards) { + this.exception = e; + this.shardEndReached = isShardEndReached; + this.childShards = childShards; + } + } diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java index ae58fb10..259b4c2f 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java @@ -30,9 +30,10 @@ public class KinesisClientLease extends Lease { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet(); - private Set childShardIds = new HashSet(); + private Set childShardIds = new HashSet<>(); private HashKeyRangeForLease hashKeyRangeForLease; + public KinesisClientLease() { } @@ -43,7 +44,7 @@ public class KinesisClientLease extends Lease { this.pendingCheckpoint = other.getPendingCheckpoint(); this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint(); this.parentShardIds.addAll(other.getParentShardIds()); - this.childShardIds = other.getChildShardIds(); + this.childShardIds.addAll(other.getChildShardIds()); this.hashKeyRangeForLease = other.getHashKeyRange(); } @@ -76,6 +77,7 @@ public class KinesisClientLease extends Lease { setCheckpoint(casted.checkpoint); setPendingCheckpoint(casted.pendingCheckpoint); setParentShardIds(casted.parentShardIds); + setChildShardIds(casted.childShardIds); } /** @@ -108,7 +110,7 @@ public class KinesisClientLease extends Lease { } /** - * @return shardIds that are the children of this lease. Used for resharding. + * @return shardIds for the child shards of the current shard. Used for resharding. */ public Set getChildShardIds() { return new HashSet(childShardIds); @@ -170,9 +172,6 @@ public class KinesisClientLease extends Lease { * @param childShardIds may not be null */ public void setChildShardIds(Collection childShardIds) { - verifyNotNull(childShardIds, "childShardIds should not be null"); - - this.childShardIds.clear(); this.childShardIds.addAll(childShardIds); } @@ -186,7 +185,7 @@ public class KinesisClientLease extends Lease { this.hashKeyRangeForLease = hashKeyRangeForLease; } - + private void verifyNotNull(Object object, String message) { if (object == null) { throw new IllegalArgumentException(message); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 6bf9bc58..0b9271be 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -43,7 +43,7 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return false; + return leaseManager.listLeases().isEmpty(); } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java index 7a125f99..a79fc9b2 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java @@ -36,6 +36,7 @@ import java.util.Collections; import java.util.Date; import java.util.List; +import com.amazonaws.services.kinesis.model.ChildShard; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -132,7 +133,7 @@ public class KinesisDataFetcherTest { } @Test - public void testadvanceIteratorTo() throws KinesisClientLibException { + public void testadvanceIteratorTo() throws Exception { IKinesisProxy kinesis = mock(IKinesisProxy.class); ICheckpoint checkpoint = mock(ICheckpoint.class); @@ -146,9 +147,13 @@ public class KinesisDataFetcherTest { GetRecordsResult outputA = new GetRecordsResult(); List recordsA = new ArrayList(); outputA.setRecords(recordsA); + outputA.setNextShardIterator("nextShardIteratorA"); + outputA.setChildShards(Collections.emptyList()); GetRecordsResult outputB = new GetRecordsResult(); List recordsB = new ArrayList(); outputB.setRecords(recordsB); + outputB.setNextShardIterator("nextShardIteratorB"); + outputB.setChildShards(Collections.emptyList()); when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqA)).thenReturn(iteratorA); when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqB)).thenReturn(iteratorB); @@ -166,7 +171,7 @@ public class KinesisDataFetcherTest { } @Test - public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() { + public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() throws Exception{ IKinesisProxy kinesis = mock(IKinesisProxy.class); KinesisDataFetcher fetcher = new KinesisDataFetcher(kinesis, SHARD_INFO); @@ -189,7 +194,7 @@ public class KinesisDataFetcherTest { } @Test - public void testGetRecordsWithResourceNotFoundException() { + public void testGetRecordsWithResourceNotFoundException() throws Exception { // Set up arguments used by proxy String nextIterator = "TestShardIterator"; int maxRecords = 100; @@ -211,11 +216,12 @@ public class KinesisDataFetcherTest { } @Test - public void testNonNullGetRecords() { + public void testNonNullGetRecords() throws Exception { String nextIterator = "TestIterator"; int maxRecords = 100; KinesisProxy mockProxy = mock(KinesisProxy.class); + when(mockProxy.getIterator(anyString(), anyString())).thenReturn("targetIterator"); doThrow(new ResourceNotFoundException("Test Exception")).when(mockProxy).get(nextIterator, maxRecords); KinesisDataFetcher dataFetcher = new KinesisDataFetcher(mockProxy, SHARD_INFO); @@ -232,17 +238,25 @@ public class KinesisDataFetcherTest { final String NEXT_ITERATOR_ONE = "NextIteratorOne"; final String NEXT_ITERATOR_TWO = "NextIteratorTwo"; when(kinesisProxy.getIterator(anyString(), anyString())).thenReturn(INITIAL_ITERATOR); - GetRecordsResult iteratorOneResults = mock(GetRecordsResult.class); - when(iteratorOneResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_ONE); + + GetRecordsResult iteratorOneResults = new GetRecordsResult(); + iteratorOneResults.setNextShardIterator(NEXT_ITERATOR_ONE); + iteratorOneResults.setChildShards(Collections.emptyList()); when(kinesisProxy.get(eq(INITIAL_ITERATOR), anyInt())).thenReturn(iteratorOneResults); - GetRecordsResult iteratorTwoResults = mock(GetRecordsResult.class); + GetRecordsResult iteratorTwoResults = new GetRecordsResult(); + iteratorTwoResults.setNextShardIterator(NEXT_ITERATOR_TWO); + iteratorTwoResults.setChildShards(Collections.emptyList()); when(kinesisProxy.get(eq(NEXT_ITERATOR_ONE), anyInt())).thenReturn(iteratorTwoResults); - when(iteratorTwoResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_TWO); - GetRecordsResult finalResult = mock(GetRecordsResult.class); + GetRecordsResult finalResult = new GetRecordsResult(); + finalResult.setNextShardIterator(null); + List childShards = new ArrayList<>(); + ChildShard childShard = new ChildShard(); + childShard.setParentShards(Collections.singletonList("parentShardId")); + childShards.add(childShard); + finalResult.setChildShards(childShards); when(kinesisProxy.get(eq(NEXT_ITERATOR_TWO), anyInt())).thenReturn(finalResult); - when(finalResult.getNextShardIterator()).thenReturn(null); KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); dataFetcher.initialize("TRIM_HORIZON", @@ -276,13 +290,14 @@ public class KinesisDataFetcherTest { } @Test - public void testRestartIterator() { + public void testRestartIterator() throws Exception{ GetRecordsResult getRecordsResult = mock(GetRecordsResult.class); - GetRecordsResult restartGetRecordsResult = new GetRecordsResult(); + GetRecordsResult restartGetRecordsResult = mock(GetRecordsResult.class); Record record = mock(Record.class); final String initialIterator = "InitialIterator"; final String nextShardIterator = "NextShardIterator"; final String restartShardIterator = "RestartIterator"; + final String restartNextShardIterator = "RestartNextIterator"; final String sequenceNumber = "SequenceNumber"; final String iteratorType = "AT_SEQUENCE_NUMBER"; KinesisProxy kinesisProxy = mock(KinesisProxy.class); @@ -292,6 +307,7 @@ public class KinesisDataFetcherTest { when(kinesisProxy.get(eq(initialIterator), eq(10))).thenReturn(getRecordsResult); when(getRecordsResult.getRecords()).thenReturn(Collections.singletonList(record)); when(getRecordsResult.getNextShardIterator()).thenReturn(nextShardIterator); + when(getRecordsResult.getChildShards()).thenReturn(Collections.emptyList()); when(record.getSequenceNumber()).thenReturn(sequenceNumber); fetcher.initialize(InitialPositionInStream.LATEST.toString(), INITIAL_POSITION_LATEST); @@ -300,6 +316,8 @@ public class KinesisDataFetcherTest { verify(kinesisProxy).get(eq(initialIterator), eq(10)); when(kinesisProxy.getIterator(eq(SHARD_ID), eq(iteratorType), eq(sequenceNumber))).thenReturn(restartShardIterator); + when(restartGetRecordsResult.getNextShardIterator()).thenReturn(restartNextShardIterator); + when(restartGetRecordsResult.getChildShards()).thenReturn(Collections.emptyList()); when(kinesisProxy.get(eq(restartShardIterator), eq(10))).thenReturn(restartGetRecordsResult); fetcher.restartIterator(); @@ -309,7 +327,7 @@ public class KinesisDataFetcherTest { } @Test (expected = IllegalStateException.class) - public void testRestartIteratorNotInitialized() { + public void testRestartIteratorNotInitialized() throws Exception { KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); dataFetcher.restartIterator(); } @@ -354,6 +372,8 @@ public class KinesisDataFetcherTest { List expectedRecords = new ArrayList(); GetRecordsResult response = new GetRecordsResult(); response.setRecords(expectedRecords); + response.setNextShardIterator("testNextShardIterator"); + response.setChildShards(Collections.emptyList()); when(kinesis.getIterator(SHARD_ID, initialPositionInStream.getTimestamp())).thenReturn(iterator); when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqNo)).thenReturn(iterator); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java index f77d3a9c..8043e0bf 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java @@ -29,6 +29,7 @@ import static org.mockito.Mockito.when; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -74,6 +75,8 @@ public class PrefetchGetRecordsCacheIntegrationTest { private IKinesisProxy proxy; @Mock private ShardInfo shardInfo; + @Mock + private KinesisClientLibLeaseCoordinator leaseCoordinator; @Before public void setup() { @@ -171,7 +174,7 @@ public class PrefetchGetRecordsCacheIntegrationTest { } @Test - public void testExpiredIteratorException() { + public void testExpiredIteratorException() throws Exception { when(dataFetcher.getRecords(eq(MAX_RECORDS_PER_CALL))).thenAnswer(new Answer() { @Override public DataFetcherResult answer(final InvocationOnMock invocationOnMock) throws Throwable { @@ -215,6 +218,8 @@ public class PrefetchGetRecordsCacheIntegrationTest { GetRecordsResult getRecordsResult = new GetRecordsResult(); getRecordsResult.setRecords(new ArrayList<>(records)); getRecordsResult.setMillisBehindLatest(1000L); + getRecordsResult.setNextShardIterator("testNextShardIterator"); + getRecordsResult.setChildShards(Collections.emptyList()); return new AdvancingResult(getRecordsResult); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java index a4336aad..b6d7769e 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java @@ -31,6 +31,7 @@ import static org.mockito.Mockito.when; import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -98,6 +99,8 @@ public class PrefetchGetRecordsCacheTest { when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResult); when(getRecordsResult.getRecords()).thenReturn(records); + when(getRecordsResult.getNextShardIterator()).thenReturn("testNextShardIterator"); + when(getRecordsResult.getChildShards()).thenReturn(Collections.emptyList()); } @Test @@ -203,7 +206,7 @@ public class PrefetchGetRecordsCacheTest { } @Test - public void testExpiredIteratorException() { + public void testExpiredIteratorException() throws Exception{ getRecordsCache.start(); when(getRecordsRetrievalStrategy.getRecords(MAX_RECORDS_PER_CALL)).thenThrow(ExpiredIteratorException.class).thenReturn(getRecordsResult); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 1cf86c4f..f040c6a6 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -39,6 +39,7 @@ import java.io.File; import java.math.BigInteger; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Date; import java.util.List; import java.util.ListIterator; @@ -245,7 +246,7 @@ public class ShardConsumerTest { @SuppressWarnings("unchecked") @Test public final void testRecordProcessorThrowable() throws Exception { - ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + ShardInfo shardInfo = new ShardInfo("s-0-0", UUID.randomUUID().toString(), null, ExtendedSequenceNumber.TRIM_HORIZON); StreamConfig streamConfig = new StreamConfig(streamProxy, 1, @@ -271,6 +272,7 @@ public class ShardConsumerTest { final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = null; + when(streamProxy.getIterator(anyString(), anyString(), anyString())).thenReturn("startingIterator"); when(leaseManager.getLease(anyString())).thenReturn(null); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(checkpoint.getCheckpointObject(anyString())).thenReturn( @@ -538,7 +540,7 @@ public class ShardConsumerTest { int numRecs = 10; BigInteger startSeqNum = BigInteger.ONE; String streamShardId = "kinesis-0-0"; - String testConcurrencyToken = "testToken"; + String testConcurrencyToken = UUID.randomUUID().toString(); List shardList = KinesisLocalFileDataCreator.createShardList(1, "kinesis-0-", startSeqNum); // Close the shard so that shutdown is called with reason terminate shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( @@ -606,8 +608,7 @@ public class ShardConsumerTest { shardSyncer, shardSyncStrategy); - when(shardSyncStrategy.onShardConsumerShutDown(shardList)).thenReturn(new TaskResult(null)); - + when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // check on parent shards Thread.sleep(50L); @@ -657,7 +658,7 @@ public class ShardConsumerTest { } assertThat(consumer.getCurrentState(), equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE)); - assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.ZOMBIE))); + assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.TERMINATE))); verify(getRecordsCache).shutdown(); @@ -681,7 +682,7 @@ public class ShardConsumerTest { int numRecs = 10; BigInteger startSeqNum = BigInteger.ONE; String streamShardId = "kinesis-0-0"; - String testConcurrencyToken = "testToken"; + String testConcurrencyToken = UUID.randomUUID().toString(); List shardList = KinesisLocalFileDataCreator.createShardList(3, "kinesis-0-", startSeqNum); // Close the shard so that shutdown is called with reason terminate shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( @@ -749,7 +750,12 @@ public class ShardConsumerTest { shardSyncer, shardSyncStrategy); - when(shardSyncStrategy.onShardConsumerShutDown(shardList)).thenReturn(new TaskResult(null)); + List parentShardIds = new ArrayList<>(); + parentShardIds.add(shardInfo.getShardId()); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(createLease(shardInfo.getShardId(), + "leaseOwner", + parentShardIds)); + when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // check on parent shards @@ -939,7 +945,7 @@ public class ShardConsumerTest { @SuppressWarnings("unchecked") @Test public final void testConsumeShardInitializedWithPendingCheckpoint() throws Exception { - ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + ShardInfo shardInfo = new ShardInfo("s-0-0", UUID.randomUUID().toString(), null, ExtendedSequenceNumber.TRIM_HORIZON); StreamConfig streamConfig = new StreamConfig(streamProxy, 1, @@ -967,6 +973,7 @@ public class ShardConsumerTest { final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = new ExtendedSequenceNumber("999"); + when(streamProxy.getIterator(anyString(), anyString(), anyString())).thenReturn("startingIterator"); when(leaseManager.getLease(anyString())).thenReturn(null); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(config.getRecordsFetcherFactory()).thenReturn(new SimpleRecordsFetcherFactory()); @@ -1125,6 +1132,14 @@ public class ShardConsumerTest { return userRecords; } + private KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey(leaseKey); + lease.setLeaseOwner(leaseOwner); + lease.setParentShardIds(parentShardIds); + return lease; + } + Matcher initializationInputMatcher(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) { return new TypeSafeMatcher() { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index 48d71f6d..208d6448 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -18,6 +18,9 @@ import java.io.File; import java.io.IOException; import java.math.BigInteger; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -29,8 +32,10 @@ import java.util.stream.Stream; import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; import com.amazonaws.services.dynamodbv2.local.embedded.DynamoDBEmbedded; +import com.amazonaws.services.kinesis.leases.impl.Lease; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.After; @@ -41,6 +46,7 @@ import org.junit.Test; import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.KinesisClientLibIOException; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ExceptionThrowingLeaseManager.ExceptionThrowingLeaseManagerMethods; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisShardSyncer.MemoizationContext; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.util.KinesisLocalFileDataCreator; @@ -58,10 +64,14 @@ import com.amazonaws.services.kinesis.model.Shard; import junit.framework.Assert; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * @@ -75,6 +85,7 @@ public class ShardSyncerTest { InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); private static final InitialPositionInStreamExtended INITIAL_POSITION_AT_TIMESTAMP = InitialPositionInStreamExtended.newInitialPositionAtTimestamp(new Date(1000L)); + private static String LEASE_OWNER = "leaseOwner"; private final boolean cleanupLeasesOfCompletedShards = true; private static final int EXPONENT = 128; AmazonDynamoDB ddbClient = DynamoDBEmbedded.create().amazonDynamoDB(); @@ -236,7 +247,7 @@ public class ShardSyncerTest { * All open and closed shards within stream's retention period should be sync'ed when lease table is empty. */ @Test - public final void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() + public final void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable1() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { List shards = constructShardListForGraphA(); @@ -263,7 +274,7 @@ public class ShardSyncerTest { * We should only create leases for shards at LATEST when lease table is not empty. */ @Test - public final void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() + public final void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable1() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { List shards = constructShardListForGraphA(); @@ -431,8 +442,11 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShard() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl(null, - Integer.MAX_VALUE, INITIAL_POSITION_TRIM_HORIZON); + Integer.MAX_VALUE, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); } /** @@ -446,12 +460,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithDeleteLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( - ExceptionThrowingLeaseManagerMethods.DELETELEASE, c, INITIAL_POSITION_TRIM_HORIZON); + ExceptionThrowingLeaseManagerMethods.DELETELEASE, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -468,12 +486,15 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithListLeasesExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( - ExceptionThrowingLeaseManagerMethods.LISTLEASES, c, INITIAL_POSITION_TRIM_HORIZON); + ExceptionThrowingLeaseManagerMethods.LISTLEASES, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -490,12 +511,15 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithCreateLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 - int maxCallingCount = 5; + int maxCallingCount = 1; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( - ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, c,INITIAL_POSITION_TRIM_HORIZON); + ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -549,8 +573,11 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShard() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl(null, - Integer.MAX_VALUE, INITIAL_POSITION_AT_TIMESTAMP); + Integer.MAX_VALUE, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); } /** @@ -564,13 +591,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithDeleteLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.DELETELEASE, - c, INITIAL_POSITION_AT_TIMESTAMP); + c, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -587,13 +617,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithListLeasesExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.LISTLEASES, - c, INITIAL_POSITION_AT_TIMESTAMP); + c, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -610,13 +643,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithCreateLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 5; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, - c, INITIAL_POSITION_AT_TIMESTAMP); + c, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -626,7 +662,7 @@ public class ShardSyncerTest { private void testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods exceptionMethod, int exceptionTime, - InitialPositionInStreamExtended position) + InitialPositionInStreamExtended position, Set expectedLeaseKeysToCreate) throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { ExtendedSequenceNumber extendedSequenceNumber = @@ -634,35 +670,17 @@ public class ShardSyncerTest { List shards = constructShardListForGraphA(); File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); dataFile.deleteOnExit(); - IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); + final IKinesisProxy kinesisProxy = spy(new KinesisLocalFileProxy(dataFile.getAbsolutePath())); + when(kinesisProxy.getShardList()).thenReturn(shards); + when(kinesisProxy.getShardListWithFilter(any())).thenReturn(getFilteredShards(shards, position)); retryCheckAndCreateLeaseForNewShards(kinesisProxy, exceptionMethod, exceptionTime, position); List newLeases = leaseManager.listLeases(); Map expectedShardIdToCheckpointMap = new HashMap(); - for (int i = 0; i < 11; i++) { - expectedShardIdToCheckpointMap.put("shardId-" + i, extendedSequenceNumber); - } - Assert.assertEquals(expectedShardIdToCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease1 : newLeases) { - ExtendedSequenceNumber expectedCheckpoint = expectedShardIdToCheckpointMap.get(lease1.getLeaseKey()); - Assert.assertNotNull(expectedCheckpoint); - Assert.assertEquals(expectedCheckpoint, lease1.getCheckpoint()); - } + expectedLeaseKeysToCreate.forEach(l -> expectedShardIdToCheckpointMap.put(l, extendedSequenceNumber)); - KinesisClientLease closedShardLease = leaseManager.getLease("shardId-0"); - closedShardLease.setCheckpoint(ExtendedSequenceNumber.SHARD_END); - leaseManager.updateLease(closedShardLease); - expectedShardIdToCheckpointMap.remove(closedShardLease.getLeaseKey()); - KinesisClientLease childShardLease = leaseManager.getLease("shardId-6"); - childShardLease.setCheckpoint(new ExtendedSequenceNumber("34290")); - leaseManager.updateLease(childShardLease); - expectedShardIdToCheckpointMap.put(childShardLease.getLeaseKey(), new ExtendedSequenceNumber("34290")); - - retryCheckAndCreateLeaseForNewShards(kinesisProxy, exceptionMethod, exceptionTime, position); - - newLeases = leaseManager.listLeases(); Assert.assertEquals(expectedShardIdToCheckpointMap.size(), newLeases.size()); for (KinesisClientLease lease1 : newLeases) { ExtendedSequenceNumber expectedCheckpoint = expectedShardIdToCheckpointMap.get(lease1.getLeaseKey()); @@ -835,193 +853,333 @@ public class ShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (3, 4, 5) + * Initial position: LATEST + * Leases to create: (2, 6) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeLatest1() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); - currentLeases.add(newLease("shardId-3")); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); - Map expectedShardIdCheckpointMap = + final Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.LATEST); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (4, 5, 7) + * Initial position: LATEST + * Leases to create: (6) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeLatest2() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - currentLeases.add(newLease("shardId-7")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); - Map expectedShardIdCheckpointMap = + final Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: LATEST + * Leases to create: (3, 4, 9, 10) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeHorizon1() { + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange3() { List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); - currentLeases.add(newLease("shardId-3")); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (4, 5, 7) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: LATEST + * Leases to create: (8) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeHorizon2() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - currentLeases.add(newLease("shardId-7")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - Map expectedShardIdCheckpointMap = + final Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) - * For shard graph B (see the construct method doc for structure). + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is + * epochs): 0 1 2 3 - shards till + * / \ | \ / + * 4 5 1 6 - shards from epoch 103 - 205 + * / \ / \ | | + * 7 8 9 10 1 6 + * shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (9, 10) + * Initial position: LATEST + * Expected leases: (1, 6, 7, 8) + */ + @Test + public final void testDetermineNewLeasesToCreateSplitMergeLatestC_PartialHashRange5() { + final List shards = constructShardListForGraphC(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: LATEST + * Leases to create: empty set + */ + @Test + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedShardIdCheckpointMap = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", + "shardId-3", "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedShardIdCheckpointMap = Collections.emptyMap(); + + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: LATEST + * Expected leases: (4, 8, 9, 10) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (6) + * Initial position: LATEST + * Expected leases: (7) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_PartialHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (5) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) * * Current leases: empty set + * Initial position: LATEST + * Expected leases: (9, 10) */ @Test - public final void testDetermineNewLeasesToCreateGraphBNoInitialLeasesTrim() { - List shards = constructShardListForGraphB(); - List currentLeases = new ArrayList(); - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - Map expectedShardIdCheckpointMap = - new HashMap(); - for (int i = 0; i < 11; i++) { - String expectedShardId = "shardId-" + i; - expectedShardIdCheckpointMap.put(expectedShardId, ExtendedSequenceNumber.TRIM_HORIZON); - } - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeLatestB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) + * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): * 0 1 2 3 4 5- shards till epoch 102 * \ / \ / | | @@ -1029,40 +1187,22 @@ public class ShardSyncerTest { * \ / | /\ * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeAtTimestamp1() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); - - currentLeases.add(newLease("shardId-3")); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) + * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): * 0 1 2 3 4 5- shards till epoch 102 * \ / \ / | | @@ -1070,72 +1210,576 @@ public class ShardSyncerTest { * \ / | /\ * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (4, 5, 7) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeAtTimestamp2() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); - - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - currentLeases.add(newLease("shardId-7")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) - * For shard graph B (see the construct method doc for structure). - * Current leases: empty set + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: TRIM_HORIZON + * Expected leases: (3, 4, 5) */ @Test - public final void testDetermineNewLeasesToCreateGraphBNoInitialLeasesAtTimestamp() { - List shards = constructShardListForGraphB(); - List currentLeases = new ArrayList(); - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - Map expectedShardIdCheckpointMap = - new HashMap(); - for (int i = 0; i < shards.size(); i++) { - String expectedShardId = "shardId-" + i; - expectedShardIdCheckpointMap.put(expectedShardId, ExtendedSequenceNumber.AT_TIMESTAMP); - } - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); } + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2, 3) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } - /* + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2, 3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (6) + * Initial position: TRIM_HORIZON + * Expected leases: (7) + */ +// TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases +// @Test +// public void testDetermineNewLeasesToCreateSplitMergeHorizonB_PartialHashRange() { +// final List shards = constructShardListForGraphB(); +// final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); +// final Map expectedShardIdCheckpointMap = new HashMap<>(); +// expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); +// testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); +// } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (5) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number)p + * + * Current leases: empty set + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (3, 4, 5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2, 3) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2, 3, 4, 5) */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (6) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (7) + */ +// TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases +// @Test +// public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_PartialHashRange() { +// final List shards = constructShardListForGraphB(); +// final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); +// final Map expectedShardIdCheckpointMap = new HashMap<>(); +// expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.AT_TIMESTAMP); +// testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); +// } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: empty set + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** * Helper method to construct a shard list for graph A. Graph A is defined below. * Shard structure (y-axis is epochs): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 210 + * \ / | /\ + * 8 4 9 10 - shards from epoch 211 (open - no ending sequenceNumber) */ List constructShardListForGraphA() { List shards = new ArrayList(); @@ -1175,14 +1819,22 @@ public class ShardSyncerTest { return shards; } - /* + /** * Helper method to construct a shard list for graph B. Graph B is defined below. * Shard structure (x-axis is epochs): - * 0 3 6 9 - * \ / \ / \ / - * 2 5 8 - * / \ / \ / \ - * 1 4 7 10 + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) */ List constructShardListForGraphB() { List shards = new ArrayList(); @@ -1214,12 +1866,54 @@ public class ShardSyncerTest { return shards; } + /** + * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is + * epochs): 0 1 2 3 - shards till + * / \ | \ / + * 4 5 1 6 - shards from epoch 103 - 205 + * / \ / \ | | + * 7 8 9 10 1 6 + * shards from epoch 206 (open - no ending sequenceNumber) + */ + private List constructShardListForGraphC() { + final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); + final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); + final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("103", null); + final SequenceNumberRange range3 = ShardObjectHelper.newSequenceNumberRange("103", "205"); + final SequenceNumberRange range4 = ShardObjectHelper.newSequenceNumberRange("206", null); + + return Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, range0, + ShardObjectHelper.newHashKeyRange("0", "399")), + ShardObjectHelper.newShard("shardId-1", null, null, range1, + ShardObjectHelper.newHashKeyRange("400", "499")), + ShardObjectHelper.newShard("shardId-2", null, null, range0, + ShardObjectHelper.newHashKeyRange("500", "599")), + ShardObjectHelper.newShard("shardId-3", null, null, range0, + ShardObjectHelper.newHashKeyRange("600", ShardObjectHelper.MAX_HASH_KEY)), + ShardObjectHelper.newShard("shardId-4", "shardId-0", null, range3, + ShardObjectHelper.newHashKeyRange("0", "199")), + ShardObjectHelper.newShard("shardId-5", "shardId-0", null, range3, + ShardObjectHelper.newHashKeyRange("200", "399")), + ShardObjectHelper.newShard("shardId-6", "shardId-2", "shardId-3", range2, + ShardObjectHelper.newHashKeyRange("500", ShardObjectHelper.MAX_HASH_KEY)), + ShardObjectHelper.newShard("shardId-7", "shardId-4", null, range4, + ShardObjectHelper.newHashKeyRange("0", "99")), + ShardObjectHelper.newShard("shardId-8", "shardId-4", null, range4, + ShardObjectHelper.newHashKeyRange("100", "199")), + ShardObjectHelper.newShard("shardId-9", "shardId-5", null, range4, + ShardObjectHelper.newHashKeyRange("200", "299")), + ShardObjectHelper.newShard("shardId-10", "shardId-5", null, range4, + ShardObjectHelper.newHashKeyRange("300", "399"))); + } + + /** * Test CheckIfDescendantAndAddNewLeasesForAncestors when shardId is null */ @Test public final void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() { - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST, null, null, @@ -1234,7 +1928,7 @@ public class ShardSyncerTest { public final void testCheckIfDescendantAndAddNewLeasesForAncestorsTrimmedShard() { String shardId = "shardId-trimmed"; Map kinesisShards = new HashMap(); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, null, kinesisShards, @@ -1253,7 +1947,7 @@ public class ShardSyncerTest { Set shardIdsOfCurrentLeases = new HashSet(); shardIdsOfCurrentLeases.add(shardId); Map newLeaseMap = new HashMap(); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertTrue(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, shardIdsOfCurrentLeases, kinesisShards, @@ -1280,7 +1974,7 @@ public class ShardSyncerTest { String shardId = "shardId-9-1"; kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null)); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, shardIdsOfCurrentLeases, kinesisShards, @@ -1309,7 +2003,7 @@ public class ShardSyncerTest { Shard shard = ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null); kinesisShards.put(shardId, shard); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertTrue(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, shardIdsOfCurrentLeases, kinesisShards, @@ -1857,6 +2551,244 @@ public class ShardSyncerTest { shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); } + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: TRIM_HORIZON + * Leases to create: (0, 1, 2, 3, 4, 5) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithEmptyLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", + "shardId-3", "shardId-4", "shardId-5")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: AT_TIMESTAMP(1000) + * Leases to create: (8, 4, 9, 10) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable1() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", + "shardId-10")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: AT_TIMESTAMP(200) + * Leases to create: (6, 7, 4, 5) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-6", "shardId-7", "shardId-4", + "shardId-5")); + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPositionAtTimestamp(new Date(200L)); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: LATEST + * Leases to create: (8, 4, 9, 10) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", + "shardId-10")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate); + } + + private void testCheckAndCreateLeaseForShardsIfMissing(List shards,InitialPositionInStreamExtended initialPositionInStreamExtended, + Set expectedLeaseKeys) throws Exception { + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPositionInStreamExtended, expectedLeaseKeys, Collections.emptyList()); + } + + private void testCheckAndCreateLeaseForShardsIfMissing(List shards,InitialPositionInStreamExtended initialPositionInStreamExtended, + Set expectedLeaseKeys, List existingLeases) throws Exception { + final File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 0, "fileName"); + dataFile.deleteOnExit(); + final IKinesisProxy kinesisProxy = spy(new KinesisLocalFileProxy(dataFile.getAbsolutePath())); + when(kinesisProxy.getShardList()).thenReturn(shards); + when(kinesisProxy.getShardListWithFilter(any())).thenReturn(getFilteredShards(shards, initialPositionInStreamExtended)); + + // Populate existing leases + for (KinesisClientLease lease : existingLeases) { + leaseManager.createLeaseIfNotExists(lease); + } + + List oldLeases = leaseManager.listLeases(); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPositionInStreamExtended, + false, false); + List newLeases = leaseManager.listLeases(); + newLeases.removeAll(oldLeases); + + final Set newLeaseKeys = newLeases.stream().map(Lease::getLeaseKey).collect(Collectors.toSet()); + final Set newSequenceNumbers = newLeases.stream().map(KinesisClientLease::getCheckpoint).collect(Collectors.toSet()); + final Set expectedSequenceNumbers = new HashSet<>(Collections + .singletonList(new ExtendedSequenceNumber(initialPositionInStreamExtended.getInitialPositionInStream().name()))); + + assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseKeys)); + assertThat(newSequenceNumbers, equalTo(expectedSequenceNumbers)); + + dataFile.delete(); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: TRIM_HORIZON + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithPartialLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from TRIM_HORIZON. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate, existingLeases); + } + + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: AT_TIMESTAMP(1000) + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable1() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from AT_TIMESTAMP. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeys= new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeys, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: AT_TIMESTAMP(200) + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPositionAtTimestamp(new Date(200L)); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from AT_TIMESTAMP. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: LATEST + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from LATEST. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate, existingLeases); + } + + + + private List createLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, + final String leaseOwner) { + return shards.stream().map(shard -> { + final Set parentShardIds = new HashSet<>(); + if (StringUtils.isNotEmpty(shard.getParentShardId())) { + parentShardIds.add(shard.getParentShardId()); + } + if (StringUtils.isNotEmpty(shard.getAdjacentParentShardId())) { + parentShardIds.add(shard.getAdjacentParentShardId()); + } + + final KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey(shard.getShardId()); + lease.setLeaseOwner(leaseOwner); + lease.setLeaseCounter(0L); + lease.setLastCounterIncrementNanos(0L); + lease.setCheckpoint(checkpoint); + lease.setOwnerSwitchesSinceCheckpoint(0L); + lease.setParentShardIds(parentShardIds); + + return lease; + }).collect(Collectors.toList()); + } /** * Helper method. * @@ -1870,6 +2802,33 @@ public class ShardSyncerTest { return lease; } + /** + * Helper method to test CheckIfDescendantAndAddNewLeasesForAncestors and verify new leases created with an expected result. + * @param shards + * @param shardIdsOfCurrentLeases + * @param checkpoint + * @param expectedShardIdCheckpointMap + */ + private void testCheckIfDescendantAndAddNewLeasesForAncestors(List shards, List shardIdsOfCurrentLeases, + InitialPositionInStreamExtended checkpoint, Map expectedShardIdCheckpointMap) { + final List currentLeases = shardIdsOfCurrentLeases.stream() + .map(shardId -> newLease(shardId)).collect(Collectors.toList()); + final Map shardIdToShardMap = KinesisShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = + KinesisShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final LeaseSynchronizer leaseSynchronizer = new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap); + final List newLeases = + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, checkpoint); + + Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); + for (KinesisClientLease lease : newLeases) { + Assert.assertTrue("Unexpected lease: " + lease, + expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); + Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); + } + } + /** * Helper method to get appropriate LeaseSynchronizer based on available shards and current leases. If there are * no current leases (empty lease table case), return EmptyLeaseTableSynchronizer. Else, return @@ -1891,4 +2850,33 @@ public class ShardSyncerTest { return new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap); } + + /** + * Helper method to mimic behavior of Kinesis ListShardsWithFilter calls. + */ + private static List getFilteredShards(List shards, InitialPositionInStreamExtended initialPosition) { + switch (initialPosition.getInitialPositionInStream()) { + case LATEST: + return shards.stream() + .filter(s -> s.getSequenceNumberRange().getEndingSequenceNumber() == null) + .collect(Collectors.toList()); + case TRIM_HORIZON: + String minSeqNum = shards.stream() + .min(Comparator.comparingLong(s -> Long.parseLong(s.getSequenceNumberRange().getStartingSequenceNumber()))) + .map(s -> s.getSequenceNumberRange().getStartingSequenceNumber()) + .orElseThrow(RuntimeException::new); + return shards.stream() + .filter(s -> s.getSequenceNumberRange().getStartingSequenceNumber().equals(minSeqNum)) + .collect(Collectors.toList()); + case AT_TIMESTAMP: + return shards.stream() + .filter(s -> new Date(Long.parseLong(s.getSequenceNumberRange().getStartingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) <= 0) + .filter(s -> s.getSequenceNumberRange().getEndingSequenceNumber() == null || + new Date(Long.parseLong(s.getSequenceNumberRange().getEndingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) > 0) + .collect(Collectors.toList()); + } + throw new RuntimeException("Unsupported initial position " + initialPosition); + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 04fadd88..cbfdf54a 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -24,11 +24,17 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.UUID; import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; import com.amazonaws.services.kinesis.model.Shard; @@ -60,7 +66,7 @@ public class ShutdownTaskTest { InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); Set defaultParentShardIds = new HashSet<>(); - String defaultConcurrencyToken = "testToken4398"; + String defaultConcurrencyToken = UUID.randomUUID().toString(); String defaultShardId = "shardId-0"; ShardInfo defaultShardInfo = new ShardInfo(defaultShardId, defaultConcurrencyToken, @@ -70,10 +76,16 @@ public class ShutdownTaskTest { ShardSyncer shardSyncer = new KinesisShardSyncer(new KinesisLeaseCleanupValidator()); + @Mock + private IKinesisProxy kinesisProxy; @Mock private GetRecordsCache getRecordsCache; @Mock private ShardSyncStrategy shardSyncStrategy; + @Mock + private ILeaseManager leaseManager; + @Mock + private KinesisClientLibLeaseCoordinator leaseCoordinator; /** * @throws java.lang.Exception @@ -95,6 +107,10 @@ public class ShutdownTaskTest { @Before public void setUp() throws Exception { doNothing().when(getRecordsCache).shutdown(); + final KinesisClientLease parentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn(parentLease); + when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); } /** @@ -111,12 +127,6 @@ public class ShutdownTaskTest { public final void testCallWhenApplicationDoesNotCheckpoint() { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - List shards = constructShardListForGraphA(); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(true, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; @@ -132,31 +142,29 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + constructChildShards()); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof IllegalArgumentException); + final String expectedExceptionMessage = "Application didn't checkpoint at end of shard shardId-0. " + + "Application must checkpoint upon shutdown. See IRecordProcessor.shutdown javadocs for more information."; + Assert.assertEquals(expectedExceptionMessage, result.getException().getMessage()); } /** * Test method for {@link ShutdownTask#call()}. */ @Test - public final void testCallWhenSyncingShardsThrows() { + public final void testCallWhenCreatingLeaseThrows() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(true, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(new KinesisClientLibIOException(""))); + final String exceptionMessage = "InvalidStateException is thrown."; + when(leaseManager.createLeaseIfNotExists(any(KinesisClientLease.class))).thenThrow(new InvalidStateException(exceptionMessage)); ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -169,30 +177,21 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + constructChildShards()); TaskResult result = task.call(); - verify(shardSyncStrategy).onShardConsumerShutDown(shards); - Assert.assertNotNull(result.getException()); - Assert.assertTrue(result.getException() instanceof KinesisClientLibIOException); verify(getRecordsCache).shutdown(); + verify(leaseCoordinator).dropLease(any(KinesisClientLease.class)); + Assert.assertNull(result.getException()); } @Test - public final void testCallWhenShardEnd() { + public final void testCallWhenShardEnd() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(true, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(null)); ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -205,36 +204,27 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + constructChildShards()); TaskResult result = task.call(); - verify(shardSyncStrategy).onShardConsumerShutDown(shards); - verify(kinesisProxy, times(1)).verifyShardClosure(anyString()); + verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); + verify(leaseCoordinator).updateLease(any(KinesisClientLease.class), any(UUID.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); - verify(leaseCoordinator, never()).dropLease(any()); } @Test - public final void testCallWhenFalseShardEnd() { + public final void testCallWhenShardNotFound() throws Exception { ShardInfo shardInfo = new ShardInfo("shardId-4", defaultConcurrencyToken, defaultParentShardIds, ExtendedSequenceNumber.LATEST); RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(false, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(new KinesisClientLease()); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(null)); - ShutdownTask task = new ShutdownTask(shardInfo, defaultRecordProcessor, checkpointer, @@ -247,31 +237,23 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + Collections.emptyList()); TaskResult result = task.call(); - verify(shardSyncStrategy, never()).onShardConsumerShutDown(shards); - verify(kinesisProxy, times(1)).verifyShardClosure(anyString()); + verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); + verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); - verify(leaseCoordinator).dropLease(any()); } @Test - public final void testCallWhenLeaseLost() { + public final void testCallWhenLeaseLost() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(false, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(null)); ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -284,13 +266,13 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + Collections.emptyList()); TaskResult result = task.call(); - verify(shardSyncStrategy, never()).onShardConsumerShutDown(shards); - verify(kinesisProxy, never()).getShardList(); + verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); + verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); - verify(leaseCoordinator, never()).dropLease(any()); } /** @@ -299,10 +281,39 @@ public class ShutdownTaskTest { @Test public final void testGetTaskType() { KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, leaseCoordinator, 0, getRecordsCache, shardSyncer, shardSyncStrategy); + ShutdownTask task = new ShutdownTask(null, null, null, null, + null, null, false, + false, leaseCoordinator, 0, + getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList()); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } + private List constructChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(defaultShardId); + + ChildShard leftChild = new ChildShard(); + leftChild.setShardId("ShardId-1"); + leftChild.setParentShards(parentShards); + leftChild.setHashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")); + childShards.add(leftChild); + + ChildShard rightChild = new ChildShard(); + rightChild.setShardId("ShardId-2"); + rightChild.setParentShards(parentShards); + rightChild.setHashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")); + childShards.add(rightChild); + return childShards; + } + + private KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey(leaseKey); + lease.setLeaseOwner(leaseOwner); + lease.setParentShardIds(parentShardIds); + return lease; + } /* * Helper method to construct a shard list for graph A. Graph A is defined below. diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java index ed57eedf..9ade05bd 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java @@ -25,6 +25,7 @@ import java.nio.charset.Charset; import java.nio.charset.CharsetEncoder; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -33,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.util.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -388,14 +390,33 @@ public class KinesisLocalFileProxy implements IKinesisProxy { */ response.setNextShardIterator(serializeIterator(iterator.shardId, lastRecordsSeqNo.add(BigInteger.ONE) .toString())); + response.setChildShards(Collections.emptyList()); LOG.debug("Returning a non null iterator for shard " + iterator.shardId); } else { + response.setChildShards(constructChildShards(iterator)); LOG.info("Returning null iterator for shard " + iterator.shardId); } return response; } + private List constructChildShards(IteratorInfo iterator) { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(iterator.shardId); + + ChildShard leftChild = new ChildShard(); + leftChild.setShardId("ShardId-1"); + leftChild.setParentShards(parentShards); + childShards.add(leftChild); + + ChildShard rightChild = new ChildShard(); + rightChild.setShardId("ShardId-2"); + rightChild.setParentShards(parentShards); + childShards.add(rightChild); + return childShards; + } + /** * {@inheritDoc} */ diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java index 4f4fdbca..d562a639 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java @@ -30,7 +30,7 @@ public class KinesisClientLeaseBuilder { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); - private Set childShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); private HashKeyRangeForLease hashKeyRangeForLease; public KinesisClientLeaseBuilder withLeaseKey(String leaseKey) { @@ -90,7 +90,6 @@ public class KinesisClientLeaseBuilder { public KinesisClientLease build() { return new KinesisClientLease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, - hashKeyRangeForLease); + checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, hashKeyRangeForLease); } } \ No newline at end of file