diff --git a/CHANGELOG.md b/CHANGELOG.md index 79937716..e3e271b3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,56 @@ # Changelog +### Latest Release (2.3.0 - August 17, 2020) + +* [Milestone#52](https://github.com/awslabs/amazon-kinesis-client/milestones/52) + +* Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. +* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. + * Currently, the following shard filters are supported: + * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. + * `AT_LATEST` - the response includes only the currently open shards of the data stream. + * `AT_TIMESTAMP` - the response includes all shards whose start timestamp is less than or equal to the given timestamp and end timestamp is greater than or equal to the given timestamp or still open. + * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `RetrievalConfig#initialPositionInStreamExtended`. + * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). + +* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. + * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. + * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). + +* KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. `PeriodicShardSyncManager` is the new component that is responsible for running periodic lease/shard scans. + * New configuration options are available to configure `PeriodicShardSyncManager` in `LeaseManagementConfig` + + | Name | Default | Description | + | ----------------------------------------------------- | ----------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | + | leasesRecoveryAuditorExecutionFrequencyMillis | 120000 (2 minutes) | Frequency (in millis) of the auditor job to scan for partial leases in the lease table. If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on leasesRecoveryAuditorInconsistencyConfidenceThreshold. | + | leasesRecoveryAuditorInconsistencyConfidenceThreshold | 3 | Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, then it would trigger a shard sync | + + * New CloudWatch metrics are also now emitted to monitor the health of `PeriodicShardSyncManager`: + + | Name | Description | + | --------------------------- | ------------------------------------------------------ | + | NumStreamsWithPartialLeases | Number of streams that had holes in their hash ranges. | + | NumStreamsToSync | Number of streams which underwent a full shard sync. | + +* Introducing deferred lease cleanup. Leases will be deleted asynchronously by `LeaseCleanupManager` upon reaching `SHARD_END`, when a shard has either expired past the stream’s retention period or been closed as the result of a resharding operation. + * New configuration options are available to configure `LeaseCleanupManager`. + + | Name | Default | Description | + | ----------------------------------- | ---------- | --------------------------------------------------------------------------------------------------------- | + | leaseCleanupIntervalMillis | 1 minute | Interval at which to run lease cleanup thread. | + | completedLeaseCleanupIntervalMillis | 5 minutes | Interval at which to check if a lease is completed or not. | + | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | + +* Introducing _experimental_ support for multistreaming, allowing a single KCL application to multiplex processing multiple streams. + * New configuration options are available to enable multistreaming in `RetrievalConfig#appStreamTracker`. + +* Fixing a bug in `PrefetchRecordsPublisher` restarting while it was already running. +* Including an optimization to `HierarchicalShardSyncer` to only create leases for one layer of shards. +* Adding support to prepare and commit lease checkpoints with arbitrary bytes. + * This allows checkpointing of an arbitrary byte buffer up to the maximum permitted DynamoDB item size ([currently 400 KB as of release](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Limits.html)), and can be used for recovery by passing a serialized byte buffer to `RecordProcessorCheckpointer#prepareCheckpoint` and `RecordProcessorCheckpointer#checkpoint`. +* Upgrading version of AWS SDK to 2.14.0. +* [#725](https://github.com/awslabs/amazon-kinesis-client/pull/725) Allowing KCL to consider lease tables in `UPDATING` healthy. + ### Release 2.2.11 (May 28, 2020) [Milestone#51](https://github.com/awslabs/amazon-kinesis-client/milestone/51) * Adjusting HTTP2 initial window size to 512 KB diff --git a/README.md b/README.md index d6b1bcf3..12a00f04 100644 --- a/README.md +++ b/README.md @@ -47,7 +47,7 @@ The recommended way to use the KCL for Java is to consume it from Maven. software.amazon.kinesis amazon-kinesis-client - 2.2.11 + 2.3.0 ``` @@ -63,14 +63,56 @@ The recommended way to use the KCL for Java is to consume it from Maven. ## Release Notes -### Release 2.2.11 (May 28, 2020) -[Milestone#51](https://github.com/awslabs/amazon-kinesis-client/milestone/51) -* Adjusting HTTP2 initial window size to 512 KB - * [PR#706](https://github.com/awslabs/amazon-kinesis-client/pull/706) -* Updating protobuf-java to version 3.11.4 - * [PR#718](https://github.com/awslabs/amazon-kinesis-client/pull/718) -* Updating the AWS Java SDK to version 2.13.25 - * [PR#722](https://github.com/awslabs/amazon-kinesis-client/pull/722) +### Latest Release (2.3.0 - August 17, 2020) + +* [Milestone#52](https://github.com/awslabs/amazon-kinesis-client/milestones/52) + +* Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. +* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. + * Currently, the following shard filters are supported: + * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. + * `AT_LATEST` - the response includes only the currently open shards of the data stream. + * `AT_TIMESTAMP` - the response includes all shards whose start timestamp is less than or equal to the given timestamp and end timestamp is greater than or equal to the given timestamp or still open. + * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `RetrievalConfig#initialPositionInStreamExtended`. + * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). + +* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. + * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. + * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). + +* KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. `PeriodicShardSyncManager` is the new component that is responsible for running periodic lease/shard scans. + * New configuration options are available to configure `PeriodicShardSyncManager` in `LeaseManagementConfig` + + | Name | Default | Description | + | ----------------------------------------------------- | ----------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | + | leasesRecoveryAuditorExecutionFrequencyMillis | 120000 (2 minutes) | Frequency (in millis) of the auditor job to scan for partial leases in the lease table. If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on leasesRecoveryAuditorInconsistencyConfidenceThreshold. | + | leasesRecoveryAuditorInconsistencyConfidenceThreshold | 3 | Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, then it would trigger a shard sync | + + * New CloudWatch metrics are also now emitted to monitor the health of `PeriodicShardSyncManager`: + + | Name | Description | + | --------------------------- | ------------------------------------------------------ | + | NumStreamsWithPartialLeases | Number of streams that had holes in their hash ranges. | + | NumStreamsToSync | Number of streams which underwent a full shard sync. | + +* Introducing deferred lease cleanup. Leases will be deleted asynchronously by `LeaseCleanupManager` upon reaching `SHARD_END`, when a shard has either expired past the stream’s retention period or been closed as the result of a resharding operation. + * New configuration options are available to configure `LeaseCleanupManager`. + + | Name | Default | Description | + | ----------------------------------- | ---------- | --------------------------------------------------------------------------------------------------------- | + | leaseCleanupIntervalMillis | 1 minute | Interval at which to run lease cleanup thread. | + | completedLeaseCleanupIntervalMillis | 5 minutes | Interval at which to check if a lease is completed or not. | + | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | + +* Introducing _experimental_ support for multistreaming, allowing a single KCL application to multiplex processing multiple streams. + * New configuration options are available to enable multistreaming in `RetrievalConfig#appStreamTracker`. + +* Fixing a bug in `PrefetchRecordsPublisher` restarting while it was already running. +* Including an optimization to `HierarchicalShardSyncer` to only create leases for one layer of shards. +* Adding support to prepare and commit lease checkpoints with arbitrary bytes. + * This allows checkpointing of an arbitrary byte buffer up to the maximum permitted DynamoDB item size ([currently 400 KB as of release](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Limits.html)), and can be used for recovery by passing a serialized byte buffer to `RecordProcessorCheckpointer#prepareCheckpoint` and `RecordProcessorCheckpointer#checkpoint`. +* Upgrading version of AWS SDK to 2.14.0. +* [#725](https://github.com/awslabs/amazon-kinesis-client/pull/725) Allowing KCL to consider lease tables in `UPDATING` healthy. ### For remaining release notes check **[CHANGELOG.md][changelog-md]**. diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 5a2654d5..ff120c94 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.2.12-SNAPSHOT + 2.3.0 4.0.0 diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java index da7e9fb2..e3368e07 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java @@ -112,6 +112,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + throw new UnsupportedOperationException(); + } + @Override public PreparedCheckpointer prepareCheckpoint(Record record) throws KinesisClientLibDependencyException, @@ -119,6 +124,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + throw new UnsupportedOperationException(); + } + @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber) throws KinesisClientLibDependencyException, @@ -126,6 +136,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + return null; + } + @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) throws KinesisClientLibDependencyException, @@ -133,6 +148,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + throw new UnsupportedOperationException(); + } + @Override public Checkpointer checkpointer() { throw new UnsupportedOperationException(); diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java index 5101243b..07f8082b 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java @@ -171,6 +171,14 @@ public class MultiLangDaemonConfigurationTest { utilsBean.setProperty(configuration, "retrievalMode", "invalid"); } + // @Test + // TODO : Enable this test once https://github.com/awslabs/amazon-kinesis-client/issues/692 is resolved + public void testmetricsEnabledDimensions() { + MultiLangDaemonConfiguration configuration = baseConfiguration(); + configuration.setMetricsEnabledDimensions(new String[]{"Operation"}); + configuration.resolvedConfiguration(shardRecordProcessorFactory); + } + @Test public void testFanoutConfigSetConsumerName() { String consumerArn = "test-consumer"; diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 708ac344..55fd63b4 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.2.12-SNAPSHOT + 2.3.0 amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java index 2bab0cd6..f5af81e3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java @@ -26,18 +26,26 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; public class Checkpoint { private final ExtendedSequenceNumber checkpoint; private final ExtendedSequenceNumber pendingCheckpoint; + private final byte[] pendingCheckpointState; + + @Deprecated + public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) { + this(checkpoint, pendingCheckpoint, null); + } /** * Constructor. * * @param checkpoint the checkpoint sequence number - cannot be null or empty. * @param pendingCheckpoint the pending checkpoint sequence number - can be null. + * @param pendingCheckpointState the pending checkpoint state - can be null. */ - public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) { + public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState) { if (checkpoint == null || checkpoint.sequenceNumber().isEmpty()) { throw new IllegalArgumentException("Checkpoint cannot be null or empty"); } this.checkpoint = checkpoint; this.pendingCheckpoint = pendingCheckpoint; + this.pendingCheckpointState = pendingCheckpointState; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java index 4705d564..fd375264 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java @@ -60,7 +60,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi public synchronized void checkpoint() throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { if (log.isDebugEnabled()) { - log.debug("Checkpointing {}, token {} at largest permitted value {}", shardInfo.shardId(), + log.debug("Checkpointing {}, token {} at largest permitted value {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), this.largestPermittedCheckpointValue); } advancePosition(this.largestPermittedCheckpointValue); @@ -116,7 +116,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi && newCheckpoint.compareTo(largestPermittedCheckpointValue) <= 0) { if (log.isDebugEnabled()) { - log.debug("Checkpointing {}, token {} at specific extended sequence number {}", shardInfo.shardId(), + log.debug("Checkpointing {}, token {} at specific extended sequence number {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), newCheckpoint); } this.advancePosition(newCheckpoint); @@ -144,8 +144,15 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi * {@inheritDoc} */ @Override - public synchronized PreparedCheckpointer prepareCheckpoint(Record record) - throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + return prepareCheckpoint(largestPermittedCheckpointValue.sequenceNumber(), applicationState); + } + + /** + * {@inheritDoc} + */ + @Override + public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { // // TODO: UserRecord Deprecation // @@ -154,10 +161,19 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } /*else if (record instanceof UserRecord) { return prepareCheckpoint(record.sequenceNumber(), ((UserRecord) record).subSequenceNumber()); } */ else { - return prepareCheckpoint(record.sequenceNumber(), 0); + return prepareCheckpoint(record.sequenceNumber(), 0, applicationState); } } + /** + * {@inheritDoc} + */ + @Override + public synchronized PreparedCheckpointer prepareCheckpoint(Record record) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + return prepareCheckpoint(record, null); + } + /** * {@inheritDoc} */ @@ -167,13 +183,30 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi return prepareCheckpoint(sequenceNumber, 0); } + /** + * {@inheritDoc} + */ + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + return prepareCheckpoint(sequenceNumber, 0, applicationState); + } + /** * {@inheritDoc} */ @Override public synchronized PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + return prepareCheckpoint(sequenceNumber, subSequenceNumber, null); + } + /** + * {@inheritDoc} + */ + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { if (subSequenceNumber < 0) { throw new IllegalArgumentException("Could not checkpoint at invalid, negative subsequence number " + subSequenceNumber); @@ -189,9 +222,9 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi if (log.isDebugEnabled()) { log.debug("Preparing checkpoint {}, token {} at specific extended sequence number {}", - shardInfo.shardId(), shardInfo.concurrencyToken(), pendingCheckpoint); + ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), pendingCheckpoint); } - return doPrepareCheckpoint(pendingCheckpoint); + return doPrepareCheckpoint(pendingCheckpoint, applicationState); } else { throw new IllegalArgumentException(String.format( "Could not prepare checkpoint at extended sequence number %s as it did not fall into acceptable " @@ -252,10 +285,10 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi if (extendedSequenceNumber != null && !extendedSequenceNumber.equals(lastCheckpointValue)) { try { if (log.isDebugEnabled()) { - log.debug("Setting {}, token {} checkpoint to {}", shardInfo.shardId(), + log.debug("Setting {}, token {} checkpoint to {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), checkpointToRecord); } - checkpointer.setCheckpoint(shardInfo.shardId(), checkpointToRecord, shardInfo.concurrencyToken()); + checkpointer.setCheckpoint(ShardInfo.getLeaseKey(shardInfo), checkpointToRecord, shardInfo.concurrencyToken()); lastCheckpointValue = checkpointToRecord; } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { @@ -290,7 +323,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi * @throws ThrottlingException * @throws ShutdownException */ - private PreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber) + private PreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { ExtendedSequenceNumber newPrepareCheckpoint = extendedSequenceNumber; @@ -308,7 +341,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } try { - checkpointer.prepareCheckpoint(shardInfo.shardId(), newPrepareCheckpoint, shardInfo.concurrencyToken()); + checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken(), applicationState); } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { throw e; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java index b0ce7675..d9646351 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java @@ -54,54 +54,59 @@ public class DynamoDBCheckpointer implements Checkpointer { private String operation; @Override - public void setCheckpoint(final String shardId, final ExtendedSequenceNumber checkpointValue, + public void setCheckpoint(final String leaseKey, final ExtendedSequenceNumber checkpointValue, final String concurrencyToken) throws KinesisClientLibException { try { - boolean wasSuccessful = setCheckpoint(shardId, checkpointValue, UUID.fromString(concurrencyToken)); + boolean wasSuccessful = setCheckpoint(leaseKey, checkpointValue, UUID.fromString(concurrencyToken)); if (!wasSuccessful) { throw new ShutdownException("Can't update checkpoint - instance doesn't hold the lease for this shard"); } } catch (ProvisionedThroughputException e) { throw new ThrottlingException("Got throttled while updating checkpoint.", e); } catch (InvalidStateException e) { - String message = "Unable to save checkpoint for shardId " + shardId; + String message = "Unable to save checkpoint for shardId " + leaseKey; log.error(message, e); throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e); } catch (DependencyException e) { - throw new KinesisClientLibDependencyException("Unable to save checkpoint for shardId " + shardId, e); + throw new KinesisClientLibDependencyException("Unable to save checkpoint for shardId " + leaseKey, e); } } @Override - public ExtendedSequenceNumber getCheckpoint(final String shardId) throws KinesisClientLibException { + public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws KinesisClientLibException { try { - return leaseRefresher.getLease(shardId).checkpoint(); + return leaseRefresher.getLease(leaseKey).checkpoint(); } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { - String message = "Unable to fetch checkpoint for shardId " + shardId; + String message = "Unable to fetch checkpoint for shardId " + leaseKey; log.error(message, e); throw new KinesisClientLibIOException(message, e); } } @Override - public Checkpoint getCheckpointObject(final String shardId) throws KinesisClientLibException { + public Checkpoint getCheckpointObject(final String leaseKey) throws KinesisClientLibException { try { - Lease lease = leaseRefresher.getLease(shardId); - log.debug("[{}] Retrieved lease => {}", shardId, lease); - return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint()); + Lease lease = leaseRefresher.getLease(leaseKey); + log.debug("[{}] Retrieved lease => {}", leaseKey, lease); + return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint(), lease.pendingCheckpointState()); } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { - String message = "Unable to fetch checkpoint for shardId " + shardId; + String message = "Unable to fetch checkpoint for shardId " + leaseKey; log.error(message, e); throw new KinesisClientLibIOException(message, e); } } @Override - public void prepareCheckpoint(final String shardId, final ExtendedSequenceNumber pendingCheckpoint, + public void prepareCheckpoint(final String leaseKey, final ExtendedSequenceNumber pendingCheckpoint, final String concurrencyToken) throws KinesisClientLibException { + prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null); + } + + @Override + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException { try { boolean wasSuccessful = - prepareCheckpoint(shardId, pendingCheckpoint, UUID.fromString(concurrencyToken)); + prepareCheckpoint(leaseKey, pendingCheckpoint, UUID.fromString(concurrencyToken), pendingCheckpointState); if (!wasSuccessful) { throw new ShutdownException( "Can't prepare checkpoint - instance doesn't hold the lease for this shard"); @@ -109,42 +114,44 @@ public class DynamoDBCheckpointer implements Checkpointer { } catch (ProvisionedThroughputException e) { throw new ThrottlingException("Got throttled while preparing checkpoint.", e); } catch (InvalidStateException e) { - String message = "Unable to prepare checkpoint for shardId " + shardId; + String message = "Unable to prepare checkpoint for shardId " + leaseKey; log.error(message, e); throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e); } catch (DependencyException e) { - throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + shardId, e); + throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + leaseKey, e); } } @VisibleForTesting - public boolean setCheckpoint(String shardId, ExtendedSequenceNumber checkpoint, UUID concurrencyToken) + public boolean setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpoint, UUID concurrencyToken) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - Lease lease = leaseCoordinator.getCurrentlyHeldLease(shardId); + Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey); if (lease == null) { log.info("Worker {} could not update checkpoint for shard {} because it does not hold the lease", - leaseCoordinator.workerIdentifier(), shardId); + leaseCoordinator.workerIdentifier(), leaseKey); return false; } lease.checkpoint(checkpoint); lease.pendingCheckpoint(null); + lease.pendingCheckpointState(null); lease.ownerSwitchesSinceCheckpoint(0L); - return leaseCoordinator.updateLease(lease, concurrencyToken, operation, shardId); + return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } - boolean prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken) + boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken, byte[] pendingCheckpointState) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - Lease lease = leaseCoordinator.getCurrentlyHeldLease(shardId); + Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey); if (lease == null) { log.info("Worker {} could not prepare checkpoint for shard {} because it does not hold the lease", - leaseCoordinator.workerIdentifier(), shardId); + leaseCoordinator.workerIdentifier(), leaseKey); return false; } lease.pendingCheckpoint(Objects.requireNonNull(pendingCheckpoint, "pendingCheckpoint should not be null")); - return leaseCoordinator.updateLease(lease, concurrencyToken, operation, shardId); + lease.pendingCheckpointState(pendingCheckpointState); + return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java index c7f56d8d..09d28495 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java @@ -15,14 +15,18 @@ package software.amazon.kinesis.common; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; import org.apache.commons.lang3.StringUtils; -import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.coordinator.CoordinatorConfig; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -30,19 +34,21 @@ import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.RetrievalConfig; /** * This Builder is useful to create all configurations for the KCL with default values. */ -@Data +@Getter @Setter @ToString @EqualsAndHashCode @Accessors(fluent = true) public class ConfigsBuilder { /** - * Name of the stream to consume records from + * Either the name of the stream to consume records from + * Or MultiStreamTracker for all the streams to consume records from */ - @NonNull - private final String streamName; + private Either appStreamTracker; + /** * Application name for the KCL Worker */ @@ -108,6 +114,52 @@ public class ConfigsBuilder { return namespace; } + /** + * Constructor to initialize ConfigsBuilder with StreamName + * @param streamName + * @param applicationName + * @param kinesisClient + * @param dynamoDBClient + * @param cloudWatchClient + * @param workerIdentifier + * @param shardRecordProcessorFactory + */ + public ConfigsBuilder(@NonNull String streamName, @NonNull String applicationName, + @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, + @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, + @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + this.appStreamTracker = Either.right(streamName); + this.applicationName = applicationName; + this.kinesisClient = kinesisClient; + this.dynamoDBClient = dynamoDBClient; + this.cloudWatchClient = cloudWatchClient; + this.workerIdentifier = workerIdentifier; + this.shardRecordProcessorFactory = shardRecordProcessorFactory; + } + + /** + * Constructor to initialize ConfigsBuilder with MultiStreamTracker + * @param multiStreamTracker + * @param applicationName + * @param kinesisClient + * @param dynamoDBClient + * @param cloudWatchClient + * @param workerIdentifier + * @param shardRecordProcessorFactory + */ + public ConfigsBuilder(@NonNull MultiStreamTracker multiStreamTracker, @NonNull String applicationName, + @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, + @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, + @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + this.appStreamTracker = Either.left(multiStreamTracker); + this.applicationName = applicationName; + this.kinesisClient = kinesisClient; + this.dynamoDBClient = dynamoDBClient; + this.cloudWatchClient = cloudWatchClient; + this.workerIdentifier = workerIdentifier; + this.shardRecordProcessorFactory = shardRecordProcessorFactory; + } + /** * Creates a new instance of CheckpointConfig * @@ -132,8 +184,7 @@ public class ConfigsBuilder { * @return LeaseManagementConfig */ public LeaseManagementConfig leaseManagementConfig() { - return new LeaseManagementConfig(tableName(), dynamoDBClient(), kinesisClient(), streamName(), - workerIdentifier()); + return new LeaseManagementConfig(tableName(), dynamoDBClient(), kinesisClient(), workerIdentifier()); } /** @@ -170,6 +221,10 @@ public class ConfigsBuilder { * @return RetrievalConfig */ public RetrievalConfig retrievalConfig() { - return new RetrievalConfig(kinesisClient(), streamName(), applicationName()); + final RetrievalConfig retrievalConfig = + appStreamTracker.map( + multiStreamTracker -> new RetrievalConfig(kinesisClient(), multiStreamTracker, applicationName()), + streamName -> new RetrievalConfig(kinesisClient(), streamName, applicationName())); + return retrievalConfig; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java index db0393e1..37eea1a3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java @@ -28,11 +28,11 @@ public class DiagnosticUtils { /** * Util for RecordPublisher to measure the event delivery latency of the executor service and take appropriate action. - * @param shardId of the shard that is having delayed delivery + * @param resourceIdentifier of the shard that is having delayed delivery * @param enqueueTimestamp of the event submitted to the executor service * @param log Slf4j Logger from RecordPublisher to log the events */ - public static void takeDelayedDeliveryActionIfRequired(String shardId, Instant enqueueTimestamp, Logger log) { + public static void takeDelayedDeliveryActionIfRequired(String resourceIdentifier, Instant enqueueTimestamp, Logger log) { final long durationBetweenEnqueueAndAckInMillis = Duration .between(enqueueTimestamp, Instant.now()).toMillis(); if (durationBetweenEnqueueAndAckInMillis > MAX_TIME_BETWEEN_REQUEST_RESPONSE / 3) { @@ -41,9 +41,9 @@ public class DiagnosticUtils { "{}: Record delivery time to shard consumer is high at {} millis. Check the ExecutorStateEvent logs" + " to see the state of the executor service. Also check if the RecordProcessor's processing " + "time is high. ", - shardId, durationBetweenEnqueueAndAckInMillis); + resourceIdentifier, durationBetweenEnqueueAndAckInMillis); } else if (log.isDebugEnabled()) { - log.debug("{}: Record delivery time to shard consumer is {} millis", shardId, + log.debug("{}: Record delivery time to shard consumer is {} millis", resourceIdentifier, durationBetweenEnqueueAndAckInMillis); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java new file mode 100644 index 00000000..d2540073 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java @@ -0,0 +1,84 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import lombok.NonNull; +import lombok.Value; +import lombok.experimental.Accessors; +import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.services.kinesis.model.HashKeyRange; + +import java.math.BigInteger; + +@Value @Accessors(fluent = true) +/** + * Lease POJO to hold the starting hashkey range and ending hashkey range of kinesis shards. + */ +public class HashKeyRangeForLease { + + private final BigInteger startingHashKey; + private final BigInteger endingHashKey; + + public HashKeyRangeForLease(BigInteger startingHashKey, BigInteger endingHashKey) { + Validate.isTrue(startingHashKey.compareTo(endingHashKey) < 0, + "StartingHashKey %s must be less than EndingHashKey %s ", startingHashKey, endingHashKey); + this.startingHashKey = startingHashKey; + this.endingHashKey = endingHashKey; + } + + /** + * Serialize the startingHashKey for persisting in external storage + * + * @return Serialized startingHashKey + */ + public String serializedStartingHashKey() { + return startingHashKey.toString(); + } + + /** + * Serialize the endingHashKey for persisting in external storage + * + * @return Serialized endingHashKey + */ + public String serializedEndingHashKey() { + return endingHashKey.toString(); + } + + /** + * Deserialize from serialized hashKeyRange string from external storage. + * + * @param startingHashKeyStr + * @param endingHashKeyStr + * @return HashKeyRangeForLease + */ + public static HashKeyRangeForLease deserialize(@NonNull String startingHashKeyStr, @NonNull String endingHashKeyStr) { + final BigInteger startingHashKey = new BigInteger(startingHashKeyStr); + final BigInteger endingHashKey = new BigInteger(endingHashKeyStr); + Validate.isTrue(startingHashKey.compareTo(endingHashKey) < 0, + "StartingHashKey %s must be less than EndingHashKey %s ", startingHashKeyStr, endingHashKeyStr); + return new HashKeyRangeForLease(startingHashKey, endingHashKey); + } + + /** + * Construct HashKeyRangeForLease from Kinesis HashKeyRange + * + * @param hashKeyRange + * @return HashKeyRangeForLease + */ + public static HashKeyRangeForLease fromHashKeyRange(HashKeyRange hashKeyRange) { + return deserialize(hashKeyRange.startingHashKey(), hashKeyRange.endingHashKey()); + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java index 437abf28..b3bedd88 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java @@ -14,6 +14,7 @@ */ package software.amazon.kinesis.common; +import lombok.EqualsAndHashCode; import lombok.ToString; import java.util.Date; @@ -22,7 +23,7 @@ import java.util.Date; * Class that houses the entities needed to specify the position in the stream from where a new application should * start. */ -@ToString +@ToString @EqualsAndHashCode public class InitialPositionInStreamExtended { private final InitialPositionInStream position; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java new file mode 100644 index 00000000..b2582d45 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java @@ -0,0 +1,41 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import lombok.Builder; +import lombok.Getter; +import lombok.experimental.Accessors; + +/** + * Configuration for lease cleanup. + */ +@Builder +@Getter +@Accessors(fluent=true) +public class LeaseCleanupConfig { + /** + * Interval at which to run lease cleanup thread. + */ + private final long leaseCleanupIntervalMillis; + /** + * Interval at which to check if a lease is completed or not. + */ + private final long completedLeaseCleanupIntervalMillis; + /** + * Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. + */ + private final long garbageLeaseCleanupIntervalMillis; +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java index ca14155e..9f511123 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.common; import lombok.experimental.Accessors; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java new file mode 100644 index 00000000..b1057f13 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java @@ -0,0 +1,29 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import lombok.Data; +import lombok.experimental.Accessors; + +@Data +@Accessors(fluent = true) +public class StreamConfig { + private final StreamIdentifier streamIdentifier; + private final InitialPositionInStreamExtended initialPositionInStreamExtended; + private String consumerArn; +} + + diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java new file mode 100644 index 00000000..1259a609 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -0,0 +1,88 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import com.google.common.base.Joiner; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.experimental.Accessors; +import software.amazon.awssdk.utils.Validate; + +import java.util.Optional; +import java.util.regex.Pattern; + +@EqualsAndHashCode @Getter @Accessors(fluent = true) +public class StreamIdentifier { + private final Optional accountIdOptional; + private final String streamName; + private final Optional streamCreationEpochOptional; + + private static final String DELIMITER = ":"; + private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*"); + + private StreamIdentifier(@NonNull String accountId, @NonNull String streamName, @NonNull Long streamCreationEpoch) { + this.accountIdOptional = Optional.of(accountId); + this.streamName = streamName; + this.streamCreationEpochOptional = Optional.of(streamCreationEpoch); + } + + private StreamIdentifier(@NonNull String streamName) { + this.accountIdOptional = Optional.empty(); + this.streamName = streamName; + this.streamCreationEpochOptional = Optional.empty(); + } + + /** + * Serialize the current StreamIdentifier instance. + * @return + */ + public String serialize() { + return accountIdOptional.isPresent() ? + Joiner.on(DELIMITER).join(accountIdOptional.get(), streamName, streamCreationEpochOptional.get()) : + streamName; + } + + @Override + public String toString() { + return serialize(); + } + + /** + * Create a multi stream instance for StreamIdentifier from serialized stream identifier. + * The serialized stream identifier should be of the format account:stream:creationepoch + * @param streamIdentifierSer + * @return StreamIdentifier + */ + public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) { + if (PATTERN.matcher(streamIdentifierSer).matches()) { + final String[] split = streamIdentifierSer.split(DELIMITER); + return new StreamIdentifier(split[0], split[1], Long.parseLong(split[2])); + } else { + throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer); + } + } + + /** + * Create a single stream instance for StreamIdentifier from stream name. + * @param streamName + * @return StreamIdentifier + */ + public static StreamIdentifier singleStreamInstance(String streamName) { + Validate.notEmpty(streamName, "StreamName should not be empty"); + return new StreamIdentifier(streamName); + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java new file mode 100644 index 00000000..720103c6 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java @@ -0,0 +1,152 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; + +import java.time.Instant; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.BooleanSupplier; +import java.util.stream.Collectors; + +/** + * An implementation of the {@code LeaderDecider} to elect leader(s) based on workerId. + * Leases are shuffled using a predetermined constant seed so that lease ordering is + * preserved across workers. + * This reduces the probability of choosing the leader workers co-located on the same + * host in case workerId starts with a common string (e.g. IP Address). + * Hence if a host has 3 workers, IPADDRESS_Worker1, IPADDRESS_Worker2, and IPADDRESS_Worker3, + * we don't end up choosing all 3 for shard sync as a result of natural ordering of Strings. + * This ensures redundancy for shard-sync during host failures. + */ +@Slf4j +class DeterministicShuffleShardSyncLeaderDecider + implements LeaderDecider { + // Fixed seed so that the shuffle order is preserved across workers + static final int DETERMINISTIC_SHUFFLE_SEED = 1947; + + private static final long ELECTION_INITIAL_DELAY_MILLIS = 60 * 1000; + private static final long ELECTION_SCHEDULING_INTERVAL_MILLIS = 5 * 60 * 1000; + private static final int AWAIT_TERMINATION_MILLIS = 5000; + + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + + private final LeaseRefresher leaseRefresher; + private final int numPeriodicShardSyncWorkers; + private final ScheduledExecutorService leaderElectionThreadPool; + + private volatile Set leaders; + + /** + * @param leaseRefresher LeaseManager instance used to fetch leases. + * @param leaderElectionThreadPool Thread-pool to be used for leaderElection. + * @param numPeriodicShardSyncWorkers Number of leaders that will be elected to perform periodic shard syncs. + */ + DeterministicShuffleShardSyncLeaderDecider(LeaseRefresher leaseRefresher, ScheduledExecutorService leaderElectionThreadPool, + int numPeriodicShardSyncWorkers) { + this.leaseRefresher = leaseRefresher; + this.leaderElectionThreadPool = leaderElectionThreadPool; + this.numPeriodicShardSyncWorkers = numPeriodicShardSyncWorkers; + } + + /* + * Shuffles the leases deterministically and elects numPeriodicShardSyncWorkers number of workers + * as leaders (workers that will perform shard sync). + */ + private void electLeaders() { + try { + log.debug("Started leader election at: " + Instant.now()); + List leases = leaseRefresher.listLeases(); + List uniqueHosts = leases.stream().map(Lease::leaseOwner) + .filter(owner -> owner != null).distinct().sorted().collect(Collectors.toList()); + + Collections.shuffle(uniqueHosts, new Random(DETERMINISTIC_SHUFFLE_SEED)); + int numShardSyncWorkers = Math.min(uniqueHosts.size(), numPeriodicShardSyncWorkers); + // In case value is currently being read, we wait for reading to complete before updating the variable. + // This is to prevent any ConcurrentModificationException exceptions. + readWriteLock.writeLock().lock(); + leaders = new HashSet<>(uniqueHosts.subList(0, numShardSyncWorkers)); + log.info("Elected leaders: " + String.join(", ", leaders)); + log.debug("Completed leader election at: " + Instant.now()); + } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { + log.error("Exception occurred while trying to fetch all leases for leader election", e); + } catch (Throwable t) { + log.error("Unknown exception during leader election.", t); + } finally { + readWriteLock.writeLock().unlock(); + } + } + + private boolean isWorkerLeaderForShardSync(String workerId) { + return CollectionUtils.isNullOrEmpty(leaders) || leaders.contains(workerId); + } + + @Override + public synchronized Boolean isLeader(String workerId) { + // if no leaders yet, synchronously get leaders. This will happen at first Shard Sync. + if (executeConditionCheckWithReadLock(() -> CollectionUtils.isNullOrEmpty(leaders))) { + electLeaders(); + // start a scheduled executor that will periodically update leaders. + // The first run will be after a minute. + // We don't need jitter since it is scheduled with a fixed delay and time taken to scan leases + // will be different at different times and on different hosts/workers. + leaderElectionThreadPool.scheduleWithFixedDelay(this::electLeaders, ELECTION_INITIAL_DELAY_MILLIS, + ELECTION_SCHEDULING_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); + } + + return executeConditionCheckWithReadLock(() -> isWorkerLeaderForShardSync(workerId)); + } + + @Override + public synchronized void shutdown() { + try { + leaderElectionThreadPool.shutdown(); + if (leaderElectionThreadPool.awaitTermination(AWAIT_TERMINATION_MILLIS, TimeUnit.MILLISECONDS)) { + log.info("Successfully stopped leader election on the worker"); + } else { + leaderElectionThreadPool.shutdownNow(); + log.info(String.format("Stopped leader election thread after awaiting termination for %d milliseconds", + AWAIT_TERMINATION_MILLIS)); + } + + } catch (InterruptedException e) { + log.debug("Encountered InterruptedException while awaiting leader election threadPool termination"); + } + } + + // Execute condition checks using shared variables under a read-write lock. + private boolean executeConditionCheckWithReadLock(BooleanSupplier action) { + try { + readWriteLock.readLock().lock(); + return action.getAsBoolean(); + } finally { + readWriteLock.readLock().unlock(); + } + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java index 3333cc42..33c83a5c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java @@ -28,7 +28,7 @@ import java.util.concurrent.ThreadPoolExecutor; @ToString(exclude = "isThreadPoolExecutor") @Slf4j @KinesisClientInternalApi -class ExecutorStateEvent implements DiagnosticEvent { +public class ExecutorStateEvent implements DiagnosticEvent { private static final String MESSAGE = "Current thread pool executor state: "; private boolean isThreadPoolExecutor; @@ -41,6 +41,11 @@ class ExecutorStateEvent implements DiagnosticEvent { private int maximumPoolSize; ExecutorStateEvent(ExecutorService executor, LeaseCoordinator leaseCoordinator) { + this(executor); + this.leasesOwned = leaseCoordinator.getAssignments().size(); + } + + public ExecutorStateEvent(ExecutorService executor) { if (executor instanceof ThreadPoolExecutor) { this.isThreadPoolExecutor = true; @@ -52,8 +57,6 @@ class ExecutorStateEvent implements DiagnosticEvent { this.largestPoolSize = ex.getLargestPoolSize(); this.maximumPoolSize = ex.getMaximumPoolSize(); } - - this.leasesOwned = leaseCoordinator.getAssignments().size(); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java new file mode 100644 index 00000000..140791af --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java @@ -0,0 +1,39 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +/** + * Used in conjunction with periodic shard sync. + * Implement this interface to allow KCL to decide if the current worker should execute shard sync. + * When periodic shard sync is enabled, PeriodicShardSyncManager periodically checks if the current + * worker is one of the leaders designated to execute shard-sync and then acts accordingly. + */ +public interface LeaderDecider { + + /** + * Method invoked to check the given workerId corresponds to one of the workers + * designated to execute shard-syncs periodically. + * + * @param workerId ID of the worker + * @return True if the worker with ID workerId can execute shard-sync. False otherwise. + */ + Boolean isLeader(String workerId); + + /** + * Can be invoked, if needed, to shutdown any clients/thread-pools + * being used in the LeaderDecider implementation. + */ + void shutdown(); +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java new file mode 100644 index 00000000..b4999bec --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java @@ -0,0 +1,437 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ComparisonChain; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.Value; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.common.HashKeyRangeForLease; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.MultiStreamLease; +import software.amazon.kinesis.leases.ShardDetector; +import software.amazon.kinesis.leases.ShardSyncTaskManager; +import software.amazon.kinesis.leases.UpdateField; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; +import software.amazon.kinesis.lifecycle.TaskResult; +import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; +import software.amazon.kinesis.metrics.MetricsUtil; + +import java.io.Serializable; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRange; + +/** + * The top level orchestrator for coordinating the periodic shard sync related + * activities. + */ +@Getter +@EqualsAndHashCode +@Slf4j +class PeriodicShardSyncManager { + private static final long INITIAL_DELAY = 60 * 1000L; + @VisibleForTesting + static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; + @VisibleForTesting + static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); + static final String PERIODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager"; + private Map hashRangeHoleTrackerMap = new HashMap<>(); + + private final String workerId; + private final LeaderDecider leaderDecider; + private final LeaseRefresher leaseRefresher; + private final Map currentStreamConfigMap; + private final Function shardSyncTaskManagerProvider; + private final ScheduledExecutorService shardSyncThreadPool; + private final boolean isMultiStreamingMode; + private final MetricsFactory metricsFactory; + private final long leasesRecoveryAuditorExecutionFrequencyMillis; + private final int leasesRecoveryAuditorInconsistencyConfidenceThreshold; + private boolean isRunning; + + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher, + Map currentStreamConfigMap, + Function shardSyncTaskManagerProvider, boolean isMultiStreamingMode, + MetricsFactory metricsFactory, + long leasesRecoveryAuditorExecutionFrequencyMillis, + int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { + this(workerId, leaderDecider, leaseRefresher, currentStreamConfigMap, shardSyncTaskManagerProvider, + Executors.newSingleThreadScheduledExecutor(), isMultiStreamingMode, metricsFactory, + leasesRecoveryAuditorExecutionFrequencyMillis, leasesRecoveryAuditorInconsistencyConfidenceThreshold); + } + + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher, + Map currentStreamConfigMap, + Function shardSyncTaskManagerProvider, + ScheduledExecutorService shardSyncThreadPool, boolean isMultiStreamingMode, + MetricsFactory metricsFactory, + long leasesRecoveryAuditorExecutionFrequencyMillis, + int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { + Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); + Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); + this.workerId = workerId; + this.leaderDecider = leaderDecider; + this.leaseRefresher = leaseRefresher; + this.currentStreamConfigMap = currentStreamConfigMap; + this.shardSyncTaskManagerProvider = shardSyncTaskManagerProvider; + this.shardSyncThreadPool = shardSyncThreadPool; + this.isMultiStreamingMode = isMultiStreamingMode; + this.metricsFactory = metricsFactory; + this.leasesRecoveryAuditorExecutionFrequencyMillis = leasesRecoveryAuditorExecutionFrequencyMillis; + this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold; + } + + public synchronized TaskResult start() { + if (!isRunning) { + final Runnable periodicShardSyncer = () -> { + try { + runShardSync(); + } catch (Throwable t) { + log.error("Error during runShardSync.", t); + } + }; + shardSyncThreadPool.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, leasesRecoveryAuditorExecutionFrequencyMillis, + TimeUnit.MILLISECONDS); + isRunning = true; + + } + return new TaskResult(null); + } + + /** + * Runs shardSync once + * Does not schedule periodic shardSync + * @return the result of the task + */ + public synchronized void syncShardsOnce() throws Exception { + // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing + // TODO: for already synced streams + for(Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); + log.info("Syncing Kinesis shard info for " + streamIdentifier); + final StreamConfig streamConfig = streamConfigEntry.getValue(); + final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfig); + final TaskResult taskResult = shardSyncTaskManager.callShardSyncTask(); + if (taskResult.getException() != null) { + throw taskResult.getException(); + } + } + } + + public void stop() { + if (isRunning) { + log.info(String.format("Shutting down leader decider on worker %s", workerId)); + leaderDecider.shutdown(); + log.info(String.format("Shutting down periodic shard sync task scheduler on worker %s", workerId)); + shardSyncThreadPool.shutdown(); + isRunning = false; + } + } + + private void runShardSync() { + if (leaderDecider.isLeader(workerId)) { + log.info(String.format("WorkerId %s is leader, running the periodic shard sync task", workerId)); + + final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, + PERIODIC_SHARD_SYNC_MANAGER); + int numStreamsWithPartialLeases = 0; + int numStreamsToSync = 0; + boolean isRunSuccess = false; + final long runStartMillis = System.currentTimeMillis(); + + try { + // Construct the stream to leases map to be used in the lease sync + final Map> streamToLeasesMap = getStreamToLeasesMap( + currentStreamConfigMap.keySet()); + + // For each of the stream, check if shard sync needs to be done based on the leases state. + for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + final ShardSyncResponse shardSyncResponse = checkForShardSync(streamConfigEntry.getKey(), + streamToLeasesMap.get(streamConfigEntry.getKey())); + + numStreamsWithPartialLeases += shardSyncResponse.isHoleDetected() ? 1 : 0; + numStreamsToSync += shardSyncResponse.shouldDoShardSync ? 1 : 0; + + if (shardSyncResponse.shouldDoShardSync()) { + log.info("Periodic shard syncer initiating shard sync for {} due to the reason - {} ", + streamConfigEntry.getKey(), shardSyncResponse.reasonForDecision()); + final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider + .apply(streamConfigEntry.getValue()); + if (!shardSyncTaskManager.submitShardSyncTask()) { + log.warn( + "Failed to submit shard sync task for stream {}. This could be due to the previous pending shard sync task.", + shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + } + } else { + log.info("Skipping shard sync for {} due to the reason - {}", streamConfigEntry.getKey(), + shardSyncResponse.reasonForDecision()); + } + } + isRunSuccess = true; + } catch (Exception e) { + log.error("Caught exception while running periodic shard syncer.", e); + } finally { + scope.addData("NumStreamsWithPartialLeases", numStreamsWithPartialLeases, StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("NumStreamsToSync", numStreamsToSync, StandardUnit.COUNT, MetricsLevel.SUMMARY); + MetricsUtil.addSuccessAndLatency(scope, isRunSuccess, runStartMillis, MetricsLevel.SUMMARY); + scope.end(); + } + } else { + log.debug("WorkerId {} is not a leader, not running the shard sync task", workerId); + } + } + + private Map> getStreamToLeasesMap( + final Set streamIdentifiersToFilter) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final List leases = leaseRefresher.listLeases(); + if (!isMultiStreamingMode) { + Validate.isTrue(streamIdentifiersToFilter.size() == 1); + return Collections.singletonMap(streamIdentifiersToFilter.iterator().next(), leases); + } else { + final Map> streamToLeasesMap = new HashMap<>(); + for (Lease lease : leases) { + StreamIdentifier streamIdentifier = StreamIdentifier + .multiStreamInstance(((MultiStreamLease) lease).streamIdentifier()); + if (streamIdentifiersToFilter.contains(streamIdentifier)) { + streamToLeasesMap.computeIfAbsent(streamIdentifier, s -> new ArrayList<>()).add(lease); + } + } + return streamToLeasesMap; + } + } + + @VisibleForTesting + ShardSyncResponse checkForShardSync(StreamIdentifier streamIdentifier, List leases) { + if (CollectionUtils.isNullOrEmpty(leases)) { + // If the leases is null or empty then we need to do shard sync + log.info("No leases found for {}. Will be triggering shard sync", streamIdentifier); + return new ShardSyncResponse(true, false, "No leases found for " + streamIdentifier); + } + // Check if there are any holes in the leases and return the first hole if present. + Optional hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases); + if (hashRangeHoleOpt.isPresent()) { + // If hole is present, check if the hole is detected consecutively in previous occurrences. + // If hole is determined with high confidence return true; return false otherwise + // We are using the high confidence factor to avoid shard sync on any holes during resharding and + // lease cleanups or any intermittent issues. + final HashRangeHoleTracker hashRangeHoleTracker = hashRangeHoleTrackerMap + .computeIfAbsent(streamIdentifier, s -> new HashRangeHoleTracker()); + final boolean hasHoleWithHighConfidence = hashRangeHoleTracker + .hasHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); + return new ShardSyncResponse(hasHoleWithHighConfidence, true, + "Detected same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + + " times. Shard sync will be initiated when threshold reaches " + + leasesRecoveryAuditorInconsistencyConfidenceThreshold); + + } else { + // If hole is not present, clear any previous tracking for this stream and return false; + hashRangeHoleTrackerMap.remove(streamIdentifier); + return new ShardSyncResponse(false, false, "Hash Ranges are complete for " + streamIdentifier); + } + } + + @Value + @Accessors(fluent = true) + @VisibleForTesting + static class ShardSyncResponse { + private final boolean shouldDoShardSync; + private final boolean isHoleDetected; + private final String reasonForDecision; + } + + @VisibleForTesting + Optional hasHoleInLeases(StreamIdentifier streamIdentifier, List leases) { + // Filter the leases with any checkpoint other than shard end. + List activeLeases = leases.stream() + .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()).collect(Collectors.toList()); + List activeLeasesWithHashRanges = fillWithHashRangesIfRequired(streamIdentifier, activeLeases); + return checkForHoleInHashKeyRanges(streamIdentifier, activeLeasesWithHashRanges); + } + + // If leases are missing hashranges information, update the leases in-memory as well as in the lease storage + // by learning from kinesis shards. + private List fillWithHashRangesIfRequired(StreamIdentifier streamIdentifier, List activeLeases) { + List activeLeasesWithNoHashRanges = activeLeases.stream() + .filter(lease -> lease.hashKeyRangeForLease() == null).collect(Collectors.toList()); + Optional minLeaseOpt = activeLeasesWithNoHashRanges.stream().min(Comparator.comparing(Lease::leaseKey)); + if (minLeaseOpt.isPresent()) { + // TODO : use minLease for new ListShards with startingShardId + final Lease minLease = minLeaseOpt.get(); + final ShardDetector shardDetector = shardSyncTaskManagerProvider + .apply(currentStreamConfigMap.get(streamIdentifier)).shardDetector(); + final Map kinesisShards = shardDetector.listShards().stream() + .collect(Collectors.toMap(Shard::shardId, shard -> shard)); + return activeLeases.stream().map(lease -> { + if (lease.hashKeyRangeForLease() == null) { + final String shardId = lease instanceof MultiStreamLease ? + ((MultiStreamLease) lease).shardId() : + lease.leaseKey(); + final Shard shard = kinesisShards.get(shardId); + if(shard == null) { + return lease; + } + lease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); + try { + leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + } catch (Exception e) { + log.warn( + "Unable to update hash range key information for lease {} of stream {}. This may result in explicit lease sync.", + lease.leaseKey(), streamIdentifier); + } + } + return lease; + }).filter(lease -> lease.hashKeyRangeForLease() != null).collect(Collectors.toList()); + } else { + return activeLeases; + } + } + + @VisibleForTesting + static Optional checkForHoleInHashKeyRanges(StreamIdentifier streamIdentifier, + List leasesWithHashKeyRanges) { + // Sort the hash ranges by starting hash key. + List sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges); + if(sortedLeasesWithHashKeyRanges.isEmpty()) { + log.error("No leases with valid hashranges found for stream {}", streamIdentifier); + return Optional.of(new HashRangeHole()); + } + // Validate for hashranges bounds. + if (!sortedLeasesWithHashKeyRanges.get(0).hashKeyRangeForLease().startingHashKey().equals(MIN_HASH_KEY) || !sortedLeasesWithHashKeyRanges + .get(sortedLeasesWithHashKeyRanges.size() - 1).hashKeyRangeForLease().endingHashKey().equals(MAX_HASH_KEY)) { + log.error("Incomplete hash range found for stream {} between {} and {}.", streamIdentifier, + sortedLeasesWithHashKeyRanges.get(0), + sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1)); + return Optional.of(new HashRangeHole(sortedLeasesWithHashKeyRanges.get(0).hashKeyRangeForLease(), + sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1).hashKeyRangeForLease())); + } + // Check for any holes in the sorted hashrange intervals. + if (sortedLeasesWithHashKeyRanges.size() > 1) { + Lease leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(0); + HashKeyRangeForLease leftLeaseHashRange = leftMostLeaseToReportInCaseOfHole.hashKeyRangeForLease(); + for (int i = 1; i < sortedLeasesWithHashKeyRanges.size(); i++) { + final HashKeyRangeForLease rightLeaseHashRange = sortedLeasesWithHashKeyRanges.get(i).hashKeyRangeForLease(); + final BigInteger rangeDiff = rightLeaseHashRange.startingHashKey().subtract(leftLeaseHashRange.endingHashKey()); + // Case of overlapping leases when the rangediff is 0 or negative. + // signum() will be -1 for negative and 0 if value is 0. + // Merge the range for further tracking. + if (rangeDiff.signum() <= 0) { + leftLeaseHashRange = new HashKeyRangeForLease(leftLeaseHashRange.startingHashKey(), + leftLeaseHashRange.endingHashKey().max(rightLeaseHashRange.endingHashKey())); + } else { + // Case of non overlapping leases when rangediff is positive. signum() will be 1 for positive. + // If rangeDiff is 1, then it is a case of continuous hashrange. If not, it is a hole. + if (!rangeDiff.equals(BigInteger.ONE)) { + log.error("Incomplete hash range found for {} between {} and {}.", streamIdentifier, + leftMostLeaseToReportInCaseOfHole, sortedLeasesWithHashKeyRanges.get(i)); + return Optional.of(new HashRangeHole(leftMostLeaseToReportInCaseOfHole.hashKeyRangeForLease(), + sortedLeasesWithHashKeyRanges.get(i).hashKeyRangeForLease())); + } + leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(i); + leftLeaseHashRange = rightLeaseHashRange; + } + } + } + return Optional.empty(); + } + + @VisibleForTesting + static List sortLeasesByHashRange(List leasesWithHashKeyRanges) { + if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1) + return leasesWithHashKeyRanges; + Collections.sort(leasesWithHashKeyRanges, new HashKeyRangeComparator()); + return leasesWithHashKeyRanges; + } + + @Value + private static class HashRangeHole { + HashRangeHole() { + hashRangeAtStartOfPossibleHole = hashRangeAtEndOfPossibleHole = null; + } + + HashRangeHole(HashKeyRangeForLease hashRangeAtStartOfPossibleHole, HashKeyRangeForLease hashRangeAtEndOfPossibleHole) { + this.hashRangeAtStartOfPossibleHole = hashRangeAtStartOfPossibleHole; + this.hashRangeAtEndOfPossibleHole = hashRangeAtEndOfPossibleHole; + } + + private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole; + private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; + } + + private class HashRangeHoleTracker { + private HashRangeHole hashRangeHole; + @Getter + private Integer numConsecutiveHoles; + + public boolean hasHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) { + if (hashRangeHole.equals(this.hashRangeHole)) { + ++this.numConsecutiveHoles; + } else { + this.hashRangeHole = hashRangeHole; + this.numConsecutiveHoles = 1; + } + return numConsecutiveHoles >= leasesRecoveryAuditorInconsistencyConfidenceThreshold; + } + } + + /** + * Helper class to compare leases based on their hash range. + */ + private static class HashKeyRangeComparator implements Comparator, Serializable { + + private static final long serialVersionUID = 1L; + + @Override + public int compare(Lease lease, Lease otherLease) { + Validate.notNull(lease); + Validate.notNull(otherLease); + Validate.notNull(lease.hashKeyRangeForLease()); + Validate.notNull(otherLease.hashKeyRangeForLease()); + return ComparisonChain.start() + .compare(lease.hashKeyRangeForLease().startingHashKey(), otherLease.hashKeyRangeForLease().startingHashKey()) + .compare(lease.hashKeyRangeForLease().endingHashKey(), otherLease.hashKeyRangeForLease().endingHashKey()) + .result(); + } + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index eaeb5a1c..ed2f889c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -15,42 +15,66 @@ package software.amazon.kinesis.coordinator; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; + +import io.reactivex.plugins.RxJavaPlugins; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; - -import com.google.common.annotations.VisibleForTesting; - -import io.reactivex.plugins.RxJavaPlugins; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; +import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.LeaseSerializer; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardPrioritization; -import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.leases.ShardSyncTaskManager; -import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; +import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer; +import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.LeasingException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; import software.amazon.kinesis.lifecycle.ShardConsumerArgument; @@ -59,10 +83,14 @@ import software.amazon.kinesis.lifecycle.ShutdownNotification; import software.amazon.kinesis.lifecycle.ShutdownReason; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.CloudWatchMetricsFactory; -import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; +import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.processor.Checkpointer; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShutdownNotificationAware; @@ -70,6 +98,8 @@ import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; +import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.StreamsLeasesDeletionType; + /** * */ @@ -78,6 +108,17 @@ import software.amazon.kinesis.retrieval.RetrievalConfig; @Slf4j public class Scheduler implements Runnable { + private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1; + private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; + private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; + private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; + private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; + private static final boolean SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS = false; + private static final String MULTI_STREAM_TRACKER = "MultiStreamTracker"; + private static final String ACTIVE_STREAMS_COUNT = "ActiveStreams.Count"; + private static final String PENDING_STREAMS_DELETION_COUNT = "StreamsPendingDeletion.Count"; + private static final String DELETED_STREAMS_COUNT = "DeletedStreams.Count"; + private SchedulerLog slog = new SchedulerLog(); private final CheckpointConfig checkpointConfig; @@ -100,25 +141,32 @@ public class Scheduler implements Runnable { private final DiagnosticEventHandler diagnosticEventHandler; // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; - private final ShardSyncTaskManager shardSyncTaskManager; + private final Function shardSyncTaskManagerProvider; + private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); + private final PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager; private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; private final GracefulShutdownCoordinator gracefulShutdownCoordinator; private final WorkerStateChangeListener workerStateChangeListener; - private final InitialPositionInStreamExtended initialPosition; private final MetricsFactory metricsFactory; private final long failoverTimeMillis; private final long taskBackoffTimeMillis; - private final String streamName; + private final boolean isMultiStreamMode; + private final Map currentStreamConfigMap; + private MultiStreamTracker multiStreamTracker; + private FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; - private final ShardDetector shardDetector; + private final Function shardDetectorProvider; private final boolean ignoreUnexpetedChildShards; private final AggregatorUtil aggregatorUtil; - private final HierarchicalShardSyncer hierarchicalShardSyncer; + private final Function hierarchicalShardSyncerProvider; private final long schedulerInitializationBackoffTimeMillis; + private final LeaderDecider leaderDecider; + private final Map staleStreamDeletionMap = new HashMap<>(); + private final LeaseCleanupManager leaseCleanupManager; // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. @@ -130,6 +178,9 @@ public class Scheduler implements Runnable { private final Object lock = new Object(); + private Stopwatch streamSyncWatch = Stopwatch.createUnstarted(); + private boolean leasesSyncedOnAppInit = false; + /** * Used to ensure that only one requestedShutdown is in progress at a time. */ @@ -170,9 +221,25 @@ public class Scheduler implements Runnable { this.retrievalConfig = retrievalConfig; this.applicationName = this.coordinatorConfig.applicationName(); + this.isMultiStreamMode = this.retrievalConfig.appStreamTracker().map( + multiStreamTracker -> true, streamConfig -> false); + this.currentStreamConfigMap = this.retrievalConfig.appStreamTracker().map( + multiStreamTracker -> { + this.multiStreamTracker = multiStreamTracker; + this.formerStreamsLeasesDeletionStrategy = multiStreamTracker.formerStreamsLeasesDeletionStrategy(); + return multiStreamTracker.streamConfigList().stream() + .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); + }, + streamConfig -> + Collections.singletonMap(streamConfig.streamIdentifier(), streamConfig)); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); - this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory() + // Determine leaseSerializer based on availability of MultiStreamTracker. + final LeaseSerializer leaseSerializer = isMultiStreamMode ? + new DynamoDBMultiStreamLeaseSerializer() : + new DynamoDBLeaseSerializer(); + this.leaseCoordinator = this.leaseManagementConfig + .leaseManagementFactory(leaseSerializer, isMultiStreamMode) .createLeaseCoordinator(this.metricsFactory); this.leaseRefresher = this.leaseCoordinator.leaseRefresher(); @@ -190,9 +257,9 @@ public class Scheduler implements Runnable { this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService(); this.diagnosticEventFactory = diagnosticEventFactory; this.diagnosticEventHandler = new DiagnosticEventLogger(); - - this.shardSyncTaskManager = this.leaseManagementConfig.leaseManagementFactory() - .createShardSyncTaskManager(this.metricsFactory); + this.shardSyncTaskManagerProvider = streamConfig -> this.leaseManagementConfig + .leaseManagementFactory(leaseSerializer, isMultiStreamMode) + .createShardSyncTaskManager(this.metricsFactory, streamConfig); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -209,19 +276,26 @@ public class Scheduler implements Runnable { this.workerStateChangeListener = this.coordinatorConfig.coordinatorFactory() .createWorkerStateChangeListener(); } - this.initialPosition = retrievalConfig.initialPositionInStreamExtended(); + this.leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, + Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis(); this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis(); // this.retryGetRecordsInSeconds = this.retrievalConfig.retryGetRecordsInSeconds(); // this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool(); - this.streamName = this.retrievalConfig.streamName(); this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis(); this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts(); - this.shardDetector = this.shardSyncTaskManager.shardDetector(); + this.shardDetectorProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); - this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); + this.hierarchicalShardSyncerProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).hierarchicalShardSyncer(); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); + this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager( + leaseManagementConfig.workerIdentifier(), leaderDecider, leaseRefresher, currentStreamConfigMap, + shardSyncTaskManagerProvider, isMultiStreamMode, metricsFactory, + leaseManagementConfig.leasesRecoveryAuditorExecutionFrequencyMillis(), + leaseManagementConfig.leasesRecoveryAuditorInconsistencyConfidenceThreshold()); + this.leaseCleanupManager = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer, isMultiStreamMode) + .createLeaseCleanupManager(metricsFactory); } /** @@ -241,7 +315,6 @@ public class Scheduler implements Runnable { workerStateChangeListener.onAllInitializationAttemptsFailed(e); shutdown(); } - while (!shouldShutdown()) { runProcessLoop(); } @@ -264,28 +337,31 @@ public class Scheduler implements Runnable { log.info("Initializing LeaseCoordinator"); leaseCoordinator.initialize(); - TaskResult result = null; + TaskResult result; if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { - log.info("Syncing Kinesis shard info"); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, - metricsFactory); - result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); + if (shouldInitiateLeaseSync()) { + log.info("Worker {} is initiating the lease sync.", leaseManagementConfig.workerIdentifier()); + leaderElectedPeriodicShardSyncManager.syncShardsOnce(); + + } } else { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); } - if (result == null || result.getException() == null) { - if (!leaseCoordinator.isRunning()) { - log.info("Starting LeaseCoordinator"); - leaseCoordinator.start(); - } else { - log.info("LeaseCoordinator is already running. No need to start it."); - } - isDone = true; + leaseCleanupManager.start(); + + // If we reach this point, then we either skipped the lease sync or did not have any exception + // for any of the shard sync in the previous attempt. + if (!leaseCoordinator.isRunning()) { + log.info("Starting LeaseCoordinator"); + leaseCoordinator.start(); } else { - lastException = result.getException(); + log.info("LeaseCoordinator is already running. No need to start it."); } + log.info("Scheduling periodicShardSync"); + leaderElectedPeriodicShardSyncManager.start(); + streamSyncWatch.start(); + isDone = true; } catch (LeasingException e) { log.error("Caught exception when initializing LeaseCoordinator", e); lastException = e; @@ -296,6 +372,7 @@ public class Scheduler implements Runnable { if (!isDone) { try { Thread.sleep(schedulerInitializationBackoffTimeMillis); + leaderElectedPeriodicShardSyncManager.stop(); } catch (InterruptedException e) { log.debug("Sleep interrupted while initializing worker."); } @@ -309,30 +386,42 @@ public class Scheduler implements Runnable { } } + @VisibleForTesting + boolean shouldInitiateLeaseSync() throws InterruptedException, + DependencyException, ProvisionedThroughputException, InvalidStateException { + long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + long waitUntil = System.currentTimeMillis() + waitTime; + + boolean shouldInitiateLeaseSync = true; + while (System.currentTimeMillis() < waitUntil && (shouldInitiateLeaseSync = leaseRefresher.isLeaseTableEmpty())) { + // check every 3 seconds if lease table is still empty, + // to minimize contention between all workers bootstrapping at the same time + log.info("Lease table is still empty. Checking again in {} ms", LEASE_TABLE_CHECK_FREQUENCY_MILLIS); + Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS); + } + return shouldInitiateLeaseSync; + } + @VisibleForTesting void runProcessLoop() { try { - boolean foundCompletedShard = false; Set assignedShards = new HashSet<>(); for (ShardInfo shardInfo : getShardInfoForAssignments()) { ShardConsumer shardConsumer = createOrGetShardConsumer(shardInfo, - processorConfig.shardRecordProcessorFactory()); + processorConfig.shardRecordProcessorFactory(), leaseCleanupManager); - if (shardConsumer.isShutdown() && shardConsumer.shutdownReason().equals(ShutdownReason.SHARD_END)) { - foundCompletedShard = true; - } else { - shardConsumer.executeLifecycle(); - } + shardConsumer.executeLifecycle(); assignedShards.add(shardInfo); } - if (foundCompletedShard) { - shardSyncTaskManager.syncShardAndLeaseInfo(); - } - // clean up shard consumers for unassigned shards cleanupShardConsumers(assignedShards); + // check for new streams and sync with the scheduler state + if (isLeader()) { + checkAndSyncStreamShardsAndLeases(); + } + logExecutorState(); slog.info("Sleeping ..."); Thread.sleep(shardConsumerDispatchPollIntervalMillis); @@ -348,6 +437,218 @@ public class Scheduler implements Runnable { slog.resetInfoLogging(); } + private boolean isLeader() { + return leaderDecider.isLeader(leaseManagementConfig.workerIdentifier()); + } + + + /** + * Note: This method has package level access solely for testing purposes. + * Sync all streams method. + * @return streams that are being synced by this worker + */ + @VisibleForTesting + Set checkAndSyncStreamShardsAndLeases() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final Set streamsSynced = new HashSet<>(); + + if (shouldSyncStreamsNow()) { + final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, MULTI_STREAM_TRACKER); + + try { + + final Map newStreamConfigMap = new HashMap<>(); + final Duration waitPeriodToDeleteOldStreams = formerStreamsLeasesDeletionStrategy.waitPeriodToDeleteFormerStreams(); + // Making an immutable copy + newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() + .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); + + List leases; + + // This is done to ensure that we clean up the stale streams lingering in the lease table. + if (!leasesSyncedOnAppInit && isMultiStreamMode) { + leases = fetchMultiStreamLeases(); + syncStreamsFromLeaseTableOnAppInit(leases); + leasesSyncedOnAppInit = true; + } + + // For new streams discovered, do a shard sync and update the currentStreamConfigMap + for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.submitShardSyncTask(); + currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); + streamsSynced.add(streamIdentifier); + } else { + if (log.isDebugEnabled()) { + log.debug(streamIdentifier + " is already being processed - skipping shard sync."); + } + } + } + + final Consumer enqueueStreamLeaseDeletionOperation = streamIdentifier -> { + if (!newStreamConfigMap.containsKey(streamIdentifier)) { + staleStreamDeletionMap.putIfAbsent(streamIdentifier, Instant.now()); + } + }; + + if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) { + // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion. + // It is assumed that all the workers will always have the latest and consistent snapshot of streams + // from the multiStreamTracker. + // + // The following streams transition state among two workers are NOT considered safe, where Worker 2, on + // initialization learn about D from lease table and delete the leases for D, as it is not available + // in its latest MultiStreamTracker. + // Worker 1 : A,B,C -> A,B,C,D (latest) + // Worker 2 : BOOTS_UP -> A,B,C (stale) + // + // The following streams transition state among two workers are NOT considered safe, where Worker 2 might + // end up deleting the leases for A and D and loose progress made so far. + // Worker 1 : A,B,C -> A,B,C,D (latest) + // Worker 2 : A,B,C -> B,C (stale/partial) + // + // In order to give workers with stale stream info, sufficient time to learn about the new streams + // before attempting to delete it, we will be deferring the leases deletion based on the + // defer time period. + + currentStreamConfigMap.keySet().stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier)); + + } else if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION) { + Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiersForLeaseCleanup()).ifPresent( + streamIdentifiers -> streamIdentifiers.stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier))); + } else { + // Remove the old/stale streams identified through the new and existing streams list, without + // cleaning up their leases. Disabling deprecated shard sync + lease cleanup through a flag. + Iterator currentSetOfStreamsIter = currentStreamConfigMap.keySet().iterator(); + while (currentSetOfStreamsIter.hasNext()) { + StreamIdentifier streamIdentifier = currentSetOfStreamsIter.next(); + if (!newStreamConfigMap.containsKey(streamIdentifier)) { + if (SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS) { + log.info( + "Found old/deleted stream : {}. Triggering shard sync. Removing from tracked active streams.", streamIdentifier); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager( + currentStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.submitShardSyncTask(); + } else { + log.info( + "Found old/deleted stream : {}. Removing from tracked active streams, but not cleaning up leases," + + " as part of this workflow", streamIdentifier); + } + currentSetOfStreamsIter.remove(); + streamsSynced.add(streamIdentifier); + } + } + } + + // Now let's scan the streamIdentifiersForLeaseCleanup eligible for deferred deletion and delete them. + // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and + // the streamIdentifiersForLeaseCleanup are not present in the latest snapshot. + final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors + .partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), Collectors.toSet())); + final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier -> + Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); + final Set deletedStreamsLeases = deleteMultiStreamLeases(staleStreamIdsToBeDeleted); + streamsSynced.addAll(deletedStreamsLeases); + + // Purge the active streams from stale streams list. + final Set staleStreamIdsToBeRevived = staleStreamIdDeletionDecisionMap.get(true); + removeStreamsFromStaleStreamsList(staleStreamIdsToBeRevived); + + log.warn( + "Streams enqueued for deletion for lease table cleanup along with their scheduled time for deletion: {} ", + staleStreamDeletionMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, + entry -> entry.getValue().plus(waitPeriodToDeleteOldStreams)))); + + streamSyncWatch.reset().start(); + + MetricsUtil.addCount(metricsScope, ACTIVE_STREAMS_COUNT, newStreamConfigMap.size(), MetricsLevel.SUMMARY); + MetricsUtil.addCount(metricsScope, PENDING_STREAMS_DELETION_COUNT, staleStreamDeletionMap.size(), + MetricsLevel.SUMMARY); + MetricsUtil.addCount(metricsScope, DELETED_STREAMS_COUNT, deletedStreamsLeases.size(), MetricsLevel.SUMMARY); + } finally { + MetricsUtil.endScope(metricsScope); + } + } + return streamsSynced; + } + + @VisibleForTesting boolean shouldSyncStreamsNow() { + return isMultiStreamMode && + (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); + } + + private void syncStreamsFromLeaseTableOnAppInit(List leases) { + final Set streamIdentifiers = leases.stream() + .map(lease -> StreamIdentifier.multiStreamInstance(lease.streamIdentifier())) + .collect(Collectors.toSet()); + for (StreamIdentifier streamIdentifier : streamIdentifiers) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + currentStreamConfigMap.put(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + } + } + } + + private List fetchMultiStreamLeases() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + return (List) ((List) leaseCoordinator.leaseRefresher().listLeases()); + } + + private void removeStreamsFromStaleStreamsList(Set streamIdentifiers) { + for(StreamIdentifier streamIdentifier : streamIdentifiers) { + staleStreamDeletionMap.remove(streamIdentifier); + } + } + + private Set deleteMultiStreamLeases(Set streamIdentifiers) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final Set streamsSynced = new HashSet<>(); + List leases = null; + Map> streamIdToShardsMap = null; + for(StreamIdentifier streamIdentifier : streamIdentifiers) { + if (leases == null) { + // Lazy Load once and use many times for this iteration. + leases = fetchMultiStreamLeases(); + } + if (streamIdToShardsMap == null) { + // Lazy load once and use many times for this iteration. + streamIdToShardsMap = leases.stream().collect(Collectors + .groupingBy(MultiStreamLease::streamIdentifier, + Collectors.toCollection(ArrayList::new))); + } + log.warn("Found old/deleted stream: " + streamIdentifier + ". Directly deleting leases of this stream."); + // Deleting leases will cause the workers to shutdown the record processors for these shards. + if (deleteMultiStreamLeases(streamIdToShardsMap.get(streamIdentifier.serialize()))) { + currentStreamConfigMap.remove(streamIdentifier); + staleStreamDeletionMap.remove(streamIdentifier); + streamsSynced.add(streamIdentifier); + } + } + return streamsSynced; + } + + private boolean deleteMultiStreamLeases(List leases) { + if (leases != null) { + for (MultiStreamLease lease : leases) { + try { + leaseRefresher.deleteLease(lease); + } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { + log.error( + "Unable to delete stale stream lease {}. Skipping further deletions for this stream. Will retry later.", + lease.leaseKey(), e); + return false; + } + } + } + return true; + } + + // When a stream is no longer needed to be tracked, return a default StreamConfig with LATEST for faster shard end. + private StreamConfig getDefaultStreamConfig(StreamIdentifier streamIdentifier) { + return new StreamConfig(streamIdentifier, InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)); + } + /** * Returns whether worker can shutdown immediately. Note that this method is called from Worker's {{@link #run()} * method before every loop run, so method must do minimum amount of work to not impact shard processing timings. @@ -516,6 +817,7 @@ public class Scheduler implements Runnable { // Lost leases will force Worker to begin shutdown process for all shard consumers in // Worker.run(). leaseCoordinator.stop(); + leaderElectedPeriodicShardSyncManager.stop(); workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.SHUT_DOWN); } } @@ -549,7 +851,7 @@ public class Scheduler implements Runnable { if (!firstItem) { builder.append(", "); } - builder.append(shardInfo.shardId()); + builder.append(ShardInfo.getLeaseKey(shardInfo)); firstItem = false; } slog.info("Current stream shard assignments: " + builder.toString()); @@ -569,7 +871,8 @@ public class Scheduler implements Runnable { * @return ShardConsumer for the shard */ ShardConsumer createOrGetShardConsumer(@NonNull final ShardInfo shardInfo, - @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { + @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory, + @NonNull final LeaseCleanupManager leaseCleanupManager) { ShardConsumer consumer = shardInfoShardConsumerMap.get(shardInfo); // Instantiate a new consumer if we don't have one, or the one we // had was from an earlier @@ -578,24 +881,38 @@ public class Scheduler implements Runnable { // completely processed (shutdown reason terminate). if ((consumer == null) || (consumer.isShutdown() && consumer.shutdownReason().equals(ShutdownReason.LEASE_LOST))) { - consumer = buildConsumer(shardInfo, shardRecordProcessorFactory); + consumer = buildConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager); shardInfoShardConsumerMap.put(shardInfo, consumer); slog.infoForce("Created new shardConsumer for : " + shardInfo); } return consumer; } + private ShardSyncTaskManager createOrGetShardSyncTaskManager(StreamConfig streamConfig) { + return streamToShardSyncTaskManagerMap.computeIfAbsent(streamConfig, s -> shardSyncTaskManagerProvider.apply(s)); + } + protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, - @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { - RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory); + @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory, + @NonNull final LeaseCleanupManager leaseCleanupManager) { ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, checkpoint); + // The only case where streamName is not available will be when multistreamtracker not set. In this case, + // get the default stream name for the single stream application. + final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt()); + + // Irrespective of single stream app or multi stream app, streamConfig should always be available. + // If we have a shardInfo, that is not present in currentStreamConfigMap for whatever reason, then return default stream config + // to gracefully complete the reading. + final StreamConfig streamConfig = currentStreamConfigMap.getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + Validate.notNull(streamConfig, "StreamConfig should not be null"); + RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, streamConfig, metricsFactory); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, - streamName, + streamConfig.streamIdentifier(), leaseCoordinator, executorService, cache, - shardRecordProcessorFactory.shardRecordProcessor(), + shardRecordProcessorFactory.shardRecordProcessor(streamIdentifier), checkpoint, checkpointer, parentShardPollIntervalMillis, @@ -605,15 +922,16 @@ public class Scheduler implements Runnable { maxListShardsRetryAttempts, processorConfig.callProcessRecordsEvenForEmptyRecordList(), shardConsumerDispatchPollIntervalMillis, - initialPosition, + streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, - shardDetector, + shardDetectorProvider.apply(streamConfig), aggregatorUtil, - hierarchicalShardSyncer, - metricsFactory); + hierarchicalShardSyncerProvider.apply(streamConfig), + metricsFactory, + leaseCleanupManager); return new ShardConsumer(cache, executorService, shardInfo, lifecycleConfig.logWarningForTaskAfterMillis(), - argument, lifecycleConfig.taskExecutionListener(),lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); + argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); } /** @@ -633,7 +951,7 @@ public class Scheduler implements Runnable { ShardConsumer consumer = shardInfoShardConsumerMap.get(shard); if (consumer.leaseLost()) { shardInfoShardConsumerMap.remove(shard); - log.debug("Removed consumer for {} as lease has been lost", shard.shardId()); + log.debug("Removed consumer for {} as lease has been lost", ShardInfo.getLeaseKey(shard)); } else { consumer.executeLifecycle(); } @@ -660,6 +978,18 @@ public class Scheduler implements Runnable { executorStateEvent.accept(diagnosticEventHandler); } + private StreamIdentifier getStreamIdentifier(Optional streamIdentifierString) { + final StreamIdentifier streamIdentifier; + if (streamIdentifierString.isPresent()) { + streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierString.get()); + } else { + Validate.isTrue(!isMultiStreamMode, "Should not be in MultiStream Mode"); + streamIdentifier = this.currentStreamConfigMap.values().iterator().next().streamIdentifier(); + } + Validate.notNull(streamIdentifier, "Stream identifier should not be empty"); + return streamIdentifier; + } + /** * Logger for suppressing too much INFO logging. To avoid too much logging information Worker will output logging at * INFO level for a single pass through the main loop every minute. At DEBUG level it will output all INFO logs on diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java index 9d5f9ae2..29d6029b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java @@ -14,6 +14,7 @@ */ package software.amazon.kinesis.leases; +import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -36,6 +37,14 @@ public class DynamoUtils { return AttributeValue.builder().ss(collectionValue).build(); } + public static AttributeValue createAttributeValue(byte[] byteBufferValue) { + if (byteBufferValue == null) { + throw new IllegalArgumentException("Byte buffer attributeValues cannot be null or empty."); + } + + return AttributeValue.builder().b(SdkBytes.fromByteArray(byteBufferValue)).build(); + } + public static AttributeValue createAttributeValue(String stringValue) { if (stringValue == null || stringValue.isEmpty()) { throw new IllegalArgumentException("String attributeValues cannot be null or empty."); @@ -52,6 +61,15 @@ public class DynamoUtils { return AttributeValue.builder().n(longValue.toString()).build(); } + public static byte[] safeGetByteArray(Map dynamoRecord, String key) { + AttributeValue av = dynamoRecord.get(key); + if (av == null) { + return null; + } else { + return av.b().asByteArray(); + } + } + public static Long safeGetLong(Map dynamoRecord, String key) { AttributeValue av = dynamoRecord.get(key); if (av == null) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index 578af465..4f677524 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -17,27 +17,36 @@ package software.amazon.kinesis.leases; import java.io.Serializable; import java.math.BigInteger; import java.util.ArrayList; -import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; +import java.util.Optional; import java.util.Set; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.awssdk.services.kinesis.model.ShardFilterType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; @@ -47,6 +56,8 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRange; + /** * Helper class to sync leases with shards of the Kinesis stream. * It will create new leases/activities when it discovers new Kinesis shards (bootstrap/resharding). @@ -57,39 +68,71 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @KinesisClientInternalApi public class HierarchicalShardSyncer { + private final boolean isMultiStreamMode; + + private final String streamIdentifier; + + private static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); + private static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); + private static final int retriesForCompleteHashRange = 3; + + private static final long DELAY_BETWEEN_LIST_SHARDS_MILLIS = 1000; + + public HierarchicalShardSyncer() { + isMultiStreamMode = false; + streamIdentifier = "SingleStreamMode"; + } + + public HierarchicalShardSyncer(final boolean isMultiStreamMode, final String streamIdentifier) { + this.isMultiStreamMode = isMultiStreamMode; + this.streamIdentifier = streamIdentifier; + } + + private static final BiFunction shardIdFromLeaseDeducer = + (lease, multiStreamArgs) -> + multiStreamArgs.isMultiStreamMode() ? + ((MultiStreamLease) lease).shardId() : + lease.leaseKey(); + /** * Check and create leases for any new shards (e.g. following a reshard operation). Sync leases with Kinesis shards * (e.g. at startup, or when we reach end of a shard). + * Return true, if shard sync was performed. Return false, if shard sync is skipped. * * @param shardDetector * @param leaseRefresher * @param initialPosition - * @param cleanupLeasesOfCompletedShards - * @param ignoreUnexpectedChildShards * @param scope + * @param ignoreUnexpectedChildShards * @throws DependencyException * @throws InvalidStateException * @throws ProvisionedThroughputException * @throws KinesisClientLibIOException */ // CHECKSTYLE:OFF CyclomaticComplexity - public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, + public synchronized boolean checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, - final MetricsScope scope) throws DependencyException, InvalidStateException, - ProvisionedThroughputException, KinesisClientLibIOException { - final List latestShards = getShardList(shardDetector); - checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, cleanupLeasesOfCompletedShards, - ignoreUnexpectedChildShards, scope, latestShards); + final MetricsScope scope, final boolean ignoreUnexpectedChildShards, final boolean isLeaseTableEmpty) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { + final List latestShards = isLeaseTableEmpty ? + getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); + return checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, ignoreUnexpectedChildShards, scope, + isLeaseTableEmpty); } //Provide a pre-collcted list of shards to avoid calling ListShards API - public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, - final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards)throws DependencyException, InvalidStateException, - ProvisionedThroughputException, KinesisClientLibIOException { + public synchronized boolean checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, + final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, + List latestShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + + //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 + if (!CollectionUtils.isNullOrEmpty(latestShards)) { - log.debug("Num shards: {}", latestShards.size()); + log.debug("{} - Num shards: {}", streamIdentifier, latestShards.size()); + } else { + log.warn("Skipping shard sync for {} as no shards found from service.", streamIdentifier); + return false; } final Map shardIdToShardMap = constructShardIdToShardMap(latestShards); @@ -99,11 +142,14 @@ public class HierarchicalShardSyncer { if (!ignoreUnexpectedChildShards) { assertAllParentShardsAreClosed(inconsistentShardIds); } - - final List currentLeases = leaseRefresher.listLeases(); - - final List newLeasesToCreate = determineNewLeasesToCreate(latestShards, currentLeases, initialPosition, inconsistentShardIds); - log.debug("Num new leases to create: {}", newLeasesToCreate.size()); + final List currentLeases = isMultiStreamMode ? + leaseRefresher.listLeasesForStream(shardDetector.streamIdentifier()) : leaseRefresher.listLeases(); + final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamIdentifier()); + final LeaseSynchronizer leaseSynchronizer = isLeaseTableEmpty ? new EmptyLeaseTableSynchronizer() : + new NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + final List newLeasesToCreate = determineNewLeasesToCreate(leaseSynchronizer, latestShards, currentLeases, + initialPosition, inconsistentShardIds, multiStreamArgs); + log.debug("{} - Num new leases to create: {}", streamIdentifier, newLeasesToCreate.size()); for (Lease lease : newLeasesToCreate) { long startTime = System.currentTimeMillis(); boolean success = false; @@ -116,13 +162,8 @@ public class HierarchicalShardSyncer { } final List trackedLeases = new ArrayList<>(currentLeases); trackedLeases.addAll(newLeasesToCreate); - cleanupGarbageLeases(shardDetector, latestShards, trackedLeases, leaseRefresher); - if (cleanupLeasesOfCompletedShards) { - cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases, leaseRefresher); - } - + return true; } - // CHECKSTYLE:ON CyclomaticComplexity /** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls * and a reshard operation. @@ -154,44 +195,6 @@ public class HierarchicalShardSyncer { .flatMap(entry -> shardIdToChildShardIdsMap.get(entry.getKey()).stream()).collect(Collectors.toSet()); } - /** - * Note: this has package level access for testing purposes. - * Useful for asserting that we don't have an incomplete shard list following a reshard operation. - * We verify that if the shard is present in the shard list, it is closed and its hash key range - * is covered by its child shards. - * @param shardIdsOfClosedShards Id of the shard which is expected to be closed - * @return ShardIds of child shards (children of the expectedClosedShard) - * @throws KinesisClientLibIOException - */ - synchronized void assertClosedShardsAreCoveredOrAbsent(final Map shardIdToShardMap, - final Map> shardIdToChildShardIdsMap, final Set shardIdsOfClosedShards) - throws KinesisClientLibIOException { - final String exceptionMessageSuffix = "This can happen if we constructed the list of shards " - + " while a reshard operation was in progress."; - - for (String shardId : shardIdsOfClosedShards) { - final Shard shard = shardIdToShardMap.get(shardId); - if (shard == null) { - log.info("Shard {} is not present in Kinesis anymore.", shardId); - continue; - } - - final String endingSequenceNumber = shard.sequenceNumberRange().endingSequenceNumber(); - if (endingSequenceNumber == null) { - throw new KinesisClientLibIOException("Shard " + shardIdsOfClosedShards - + " is not closed. " + exceptionMessageSuffix); - } - - final Set childShardIds = shardIdToChildShardIdsMap.get(shardId); - if (childShardIds == null) { - throw new KinesisClientLibIOException("Incomplete shard list: Closed shard " + shardId - + " has no children." + exceptionMessageSuffix); - } - - assertHashRangeOfClosedShardIsCovered(shard, shardIdToShardMap, childShardIds); - } - } - private synchronized void assertHashRangeOfClosedShardIsCovered(final Shard closedShard, final Map shardIdToShardMap, final Set childShardIds) throws KinesisClientLibIOException { @@ -253,55 +256,118 @@ public class HierarchicalShardSyncer { return shardIdToChildShardIdsMap; } - private static List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { - final List shards = shardDetector.listShards(); - if (shards == null) { - throw new KinesisClientLibIOException( - "Stream is not in ACTIVE OR UPDATING state - will retry getting the shard list."); + /** + * Helper method to resolve the correct shard filter to use when listing shards from a position in a stream. + * @param initialPositionInStreamExtended + * @return ShardFilter shard filter for the corresponding position in the stream. + */ + private static ShardFilter getShardFilterFromInitialPosition(InitialPositionInStreamExtended initialPositionInStreamExtended) { + + ShardFilter.Builder builder = ShardFilter.builder(); + + switch (initialPositionInStreamExtended.getInitialPositionInStream()) { + case LATEST: + builder = builder.type(ShardFilterType.AT_LATEST); + break; + case TRIM_HORIZON: + builder = builder.type(ShardFilterType.AT_TRIM_HORIZON); + break; + case AT_TIMESTAMP: + builder = builder.type(ShardFilterType.AT_TIMESTAMP).timestamp(initialPositionInStreamExtended.getTimestamp().toInstant()); + break; } - return shards; + return builder.build(); + } + + private static List getShardListAtInitialPosition(@NonNull final ShardDetector shardDetector, + InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException, InterruptedException { + + final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); + final String streamName = shardDetector.streamIdentifier().streamName(); + + List shards; + + for (int i = 0; i < retriesForCompleteHashRange; i++) { + shards = shardDetector.listShardsWithFilter(shardFilter); + + if (shards == null) { + throw new KinesisClientLibIOException( + "Stream " + streamName + " is not in ACTIVE OR UPDATING state - will retry getting the shard list."); + } + + if (isHashRangeOfShardsComplete(shards)) { + return shards; + } + + Thread.sleep(DELAY_BETWEEN_LIST_SHARDS_MILLIS); + } + + throw new KinesisClientLibIOException("Hash range of shards returned for " + streamName + " was incomplete after " + + retriesForCompleteHashRange + " retries."); + } + + private static List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { + final Optional> shards = Optional.of(shardDetector.listShards()); + + return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream " + shardDetector.streamIdentifier().streamName() + + " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); + } + + private static boolean isHashRangeOfShardsComplete(@NonNull List shards) { + + if (shards.isEmpty()) { + throw new IllegalStateException("No shards found when attempting to validate complete hash range."); + } + + final Comparator shardStartingHashKeyBasedComparator = new ShardStartingHashKeyBasedComparator(); + shards.sort(shardStartingHashKeyBasedComparator); + + if (!shards.get(0).hashKeyRange().startingHashKey().equals(MIN_HASH_KEY) || + !shards.get(shards.size() - 1).hashKeyRange().endingHashKey().equals(MAX_HASH_KEY)) { + return false; + } + + if (shards.size() > 1) { + for (int i = 1; i < shards.size(); i++) { + final Shard shardAtStartOfPossibleHole = shards.get(i - 1); + final Shard shardAtEndOfPossibleHole = shards.get(i); + final BigInteger startOfPossibleHole = new BigInteger(shardAtStartOfPossibleHole.hashKeyRange().endingHashKey()); + final BigInteger endOfPossibleHole = new BigInteger(shardAtEndOfPossibleHole.hashKeyRange().startingHashKey()); + + if (!endOfPossibleHole.subtract(startOfPossibleHole).equals(BigInteger.ONE)) { + log.error("Incomplete hash range found between {} and {}.", shardAtStartOfPossibleHole, shardAtEndOfPossibleHole); + return false; + } + } + } + + return true; } /** * Determine new leases to create and their initial checkpoint. * 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. - * 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 - * we begin processing data from any of its descendants. - * * A shard does not start processing data until data from all its parents has been processed. - * Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create - * leases corresponding to both the parents - the parent shard which is not a descendant will have - * its checkpoint set to Latest. - * - * We assume that if there is an existing lease for a shard, then either: - * * we have previously created a lease for its parent (if it was needed), or - * * the parent shard has expired. - * - * For example: - * 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) - * New leases to create: (2, 6, 7, 8, 9, 10) - * - * 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 - * before creating all the leases. + * @param leaseSynchronizer determines the strategy we'll be using to update any new leases. + * @param shards List of all shards in Kinesis (we'll create new leases based on this set) + * @param currentLeases List of current leases + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that + * location in the shard (when an application starts up for the first time - and there are no checkpoints). + * @param inconsistentShardIds Set of child shard ids having open parents. + * @param multiStreamArgs determines if we are using multistream mode. + * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard + */ + static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, + final List currentLeases, final InitialPositionInStreamExtended initialPosition, + final Set inconsistentShardIds, final MultiStreamArgs multiStreamArgs) { + return leaseSynchronizer.determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds, multiStreamArgs); + } + + /** + * Determine new leases to create and their initial checkpoint. + * Note: Package level access only for testing purposes. * - * If a shard has no existing lease, is open, and is a descendant of a parent which is still open, we ignore it - * here; this happens when the list of shards is inconsistent, which could be due to pagination delay for very - * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only - * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. - * - * + * @param leaseSynchronizer determines the strategy we'll be using to update any new leases. * @param shards List of all shards in Kinesis (we'll create new leases based on this set) * @param currentLeases List of current leases * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that @@ -309,89 +375,35 @@ public class HierarchicalShardSyncer { * @param inconsistentShardIds Set of child shard ids having open parents. * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ - static List determineNewLeasesToCreate(final List shards, final List currentLeases, - final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds) { - final Map shardIdToNewLeaseMap = new HashMap<>(); - final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); - - final Set shardIdsOfCurrentLeases = currentLeases.stream() - .peek(lease -> log.debug("Existing lease: {}", lease)).map(Lease::leaseKey).collect(Collectors.toSet()); - - final List openShards = getOpenShards(shards); - final Map memoizationContext = new HashMap<>(); - - // Iterate over the open shards and find those that don't have any lease entries. - for (Shard shard : openShards) { - final String shardId = shard.shardId(); - log.debug("Evaluating leases for open shard {} and its ancestors.", shardId); - if (shardIdsOfCurrentLeases.contains(shardId)) { - log.debug("Lease for shardId {} already exists. Not creating a lease", shardId); - } else if (inconsistentShardIds.contains(shardId)) { - log.info("shardId {} is an inconsistent child. Not creating a lease", shardId); - } else { - log.debug("Need to create a lease for shardId {}", shardId); - final Lease newLease = newKCLLease(shard); - final boolean isDescendant = 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.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - newLease.checkpoint(convertToCheckpoint(initialPosition)); - } - log.debug("Set checkpoint of {} to {}", newLease.leaseKey(), newLease.checkpoint()); - shardIdToNewLeaseMap.put(shardId, newLease); - } - } - - final List newLeasesToCreate = new ArrayList<>(shardIdToNewLeaseMap.values()); - final Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMapOfAllKinesisShards); - newLeasesToCreate.sort(startingSequenceNumberComparator); - return newLeasesToCreate; + static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, + final List currentLeases, final InitialPositionInStreamExtended initialPosition,final Set inconsistentShardIds) { + return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds, + new MultiStreamArgs(false, null)); } /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. + * + * @param leaseSynchronizer determines the strategy we'll be using to update any new leases. + * @param shards List of all shards in Kinesis (we'll create new leases based on this set) + * @param currentLeases List of current leases + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that + * location in the shard (when an application starts up for the first time - and there are no checkpoints). + * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ - static List determineNewLeasesToCreate(final List shards, final List currentLeases, - final InitialPositionInStreamExtended initialPosition) { + static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, + final List currentLeases, final InitialPositionInStreamExtended initialPosition) { final Set inconsistentShardIds = new HashSet<>(); - return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds); + return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds); } /** * 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. * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * location in the shard (when an application starts up for the first time - and there are no checkpoints). @@ -405,9 +417,10 @@ public class HierarchicalShardSyncer { static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, final Map shardIdToShardMapOfAllKinesisShards, - final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext) { - - final Boolean previousValue = memoizationContext.get(shardId); + final Map shardIdToLeaseMapOfNewShards, final MemoizationContext memoizationContext, + final MultiStreamArgs multiStreamArgs) { + final String streamIdentifier = getStreamIdentifier(multiStreamArgs); + final Boolean previousValue = memoizationContext.isDescendant(shardId); if (previousValue != null) { return previousValue; } @@ -422,18 +435,22 @@ public class HierarchicalShardSyncer { // We don't need to add leases of its ancestors, // because we'd have done it when creating a lease for this shard. } else { + final Shard shard = shardIdToShardMapOfAllKinesisShards.get(shardId); final Set 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, multiStreamArgs); + if (isParentDescendant || memoizationContext.shouldCreateLease(parentShardId)) { isDescendant = true; descendantParentShardIds.add(parentShardId); - log.debug("Parent shard {} is a descendant.", parentShardId); + log.debug("{} : Parent shard {} is a descendant.", streamIdentifier, parentShardId); } else { - log.debug("Parent shard {} is NOT a descendant.", parentShardId); + log.debug("{} : Parent shard {} is NOT a descendant.", streamIdentifier, parentShardId); } } @@ -441,40 +458,92 @@ public class HierarchicalShardSyncer { if (isDescendant) { for (String parentShardId : parentShardIds) { if (!shardIdsOfCurrentLeases.contains(parentShardId)) { - log.debug("Need to create a lease for shardId {}", parentShardId); Lease 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 {}", streamIdentifier, parentShardId); + lease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId), + multiStreamArgs.streamIdentifier()) : + newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); + shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + } } - if (descendantParentShardIds.contains(parentShardId) - && !initialPosition.getInitialPositionInStream() + /** + * 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.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - lease.checkpoint(convertToCheckpoint(initialPosition)); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } else { + lease.checkpoint(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; } + + static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, + final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, + final Map shardIdToShardMapOfAllKinesisShards, + final Map shardIdToLeaseMapOfNewShards, MemoizationContext memoizationContext) { + return checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, + shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, memoizationContext, + new MultiStreamArgs(false, null)); + } + // CHECKSTYLE:ON CyclomaticComplexity /** @@ -500,165 +569,41 @@ public class HierarchicalShardSyncer { return parentShardIds; } - /** - * Delete leases corresponding to shards that no longer exist in the stream. Current scheme: Delete a lease if: - *
    - *
  • The corresponding shard is not present in the list of Kinesis shards
  • - *
  • The parentShardIds listed in the lease are also not present in the list of Kinesis shards.
  • - *
- * - * @param shards - * List of all Kinesis shards (assumed to be a consistent snapshot - when stream is in Active state). - * @param trackedLeases - * List of - * @param leaseRefresher - * @throws KinesisClientLibIOException - * Thrown if we couldn't get a fresh shard list from Kinesis. - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - */ - private static void cleanupGarbageLeases(@NonNull final ShardDetector shardDetector, final List shards, - final List trackedLeases, final LeaseRefresher leaseRefresher) throws KinesisClientLibIOException, - DependencyException, InvalidStateException, ProvisionedThroughputException { - final Set kinesisShards = shards.stream().map(Shard::shardId).collect(Collectors.toSet()); + public synchronized Lease createLeaseForChildShard(final ChildShard childShard, final StreamIdentifier streamIdentifier) throws InvalidStateException { + final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, streamIdentifier); - // Check if there are leases for non-existent shards - final List garbageLeases = trackedLeases.stream() - .filter(lease -> isCandidateForCleanup(lease, kinesisShards)).collect(Collectors.toList()); - - if (!CollectionUtils.isNullOrEmpty(garbageLeases)) { - log.info("Found {} candidate leases for cleanup. Refreshing list of" - + " Kinesis shards to pick up recent/latest shards", garbageLeases.size()); - final Set currentKinesisShardIds = getShardList(shardDetector).stream().map(Shard::shardId) - .collect(Collectors.toSet()); - - for (Lease lease : garbageLeases) { - if (isCandidateForCleanup(lease, currentKinesisShardIds)) { - log.info("Deleting lease for shard {} as it is not present in Kinesis stream.", lease.leaseKey()); - leaseRefresher.deleteLease(lease); - } - } - } + return multiStreamArgs.isMultiStreamMode() ? newKCLMultiStreamLeaseForChildShard(childShard, streamIdentifier) + : newKCLLeaseForChildShard(childShard); } - /** - * Note: This method has package level access, solely for testing purposes. - * - * @param lease Candidate shard we are considering for deletion. - * @param currentKinesisShardIds - * @return true if neither the shard (corresponding to the lease), nor its parents are present in - * currentKinesisShardIds - * @throws KinesisClientLibIOException Thrown if currentKinesisShardIds contains a parent shard but not the child - * shard (we are evaluating for deletion). - */ - static boolean isCandidateForCleanup(final Lease lease, final Set currentKinesisShardIds) - throws KinesisClientLibIOException { - boolean isCandidateForCleanup = true; - - if (currentKinesisShardIds.contains(lease.leaseKey())) { - isCandidateForCleanup = false; + private static Lease newKCLLeaseForChildShard(final ChildShard childShard) throws InvalidStateException { + Lease newLease = new Lease(); + newLease.leaseKey(childShard.shardId()); + if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + newLease.parentShardIds(childShard.parentShards()); } else { - log.info("Found lease for non-existent shard: {}. Checking its parent shards", lease.leaseKey()); - final Set parentShardIds = lease.parentShardIds(); - for (String parentShardId : parentShardIds) { - - // Throw an exception if the parent shard exists (but the child does not). - // This may be a (rare) race condition between fetching the shard list and Kinesis expiring shards. - if (currentKinesisShardIds.contains(parentShardId)) { - final String message = String.format("Parent shard %s exists but not the child shard %s", - parentShardId, lease.leaseKey()); - log.info(message); - throw new KinesisClientLibIOException(message); - } - } + throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found."); } - - return isCandidateForCleanup; - } - - /** - * Private helper method. - * Clean up leases for shards that meet the following criteria: - * a/ the shard has been fully processed (checkpoint is set to SHARD_END) - * b/ we've begun processing all the child shards: we have leases for all child shards and their checkpoint is not - * TRIM_HORIZON. - * - * @param currentLeases List of leases we evaluate for clean up - * @param shardIdToShardMap Map of shardId->Shard (assumed to include all Kinesis shards) - * @param shardIdToChildShardIdsMap Map of shardId->childShardIds (assumed to include all Kinesis shards) - * @param trackedLeases List of all leases we are tracking. - * @param leaseRefresher Lease refresher (will be used to delete leases) - * @throws DependencyException - * @throws InvalidStateException - * @throws ProvisionedThroughputException - * @throws KinesisClientLibIOException - */ - private synchronized void cleanupLeasesOfFinishedShards(final Collection currentLeases, - final Map shardIdToShardMap, final Map> shardIdToChildShardIdsMap, - final List trackedLeases, final LeaseRefresher leaseRefresher) throws DependencyException, - InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - final List leasesOfClosedShards = currentLeases.stream() - .filter(lease -> lease.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) - .collect(Collectors.toList()); - final Set shardIdsOfClosedShards = leasesOfClosedShards.stream().map(Lease::leaseKey) - .collect(Collectors.toSet()); - - if (!CollectionUtils.isNullOrEmpty(leasesOfClosedShards)) { - assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, shardIdsOfClosedShards); - Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMap); - leasesOfClosedShards.sort(startingSequenceNumberComparator); - final Map trackedLeaseMap = trackedLeases.stream() - .collect(Collectors.toMap(Lease::leaseKey, Function.identity())); - - for (Lease leaseOfClosedShard : leasesOfClosedShards) { - final String closedShardId = leaseOfClosedShard.leaseKey(); - final Set childShardIds = shardIdToChildShardIdsMap.get(closedShardId); - if (closedShardId != null && !CollectionUtils.isNullOrEmpty(childShardIds)) { - cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseRefresher); - } - } - } + newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.hashKeyRange(fromHashKeyRange(childShard.hashKeyRange())); + return newLease; } - /** - * Delete lease for the closed shard. Rules for deletion are: - * a/ the checkpoint for the closed shard is SHARD_END, - * b/ there are leases for all the childShardIds and their checkpoint is NOT TRIM_HORIZON - * Note: This method has package level access solely for testing purposes. - * - * @param closedShardId Identifies the closed shard - * @param childShardIds ShardIds of children of the closed shard - * @param trackedLeases shardId->Lease map with all leases we are tracking (should not be null) - * @param leaseRefresher - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - */ - synchronized void cleanupLeaseForClosedShard(final String closedShardId, final Set childShardIds, - final Map trackedLeases, final LeaseRefresher leaseRefresher) - throws DependencyException, InvalidStateException, ProvisionedThroughputException { - final Lease leaseForClosedShard = trackedLeases.get(closedShardId); - final List childShardLeases = childShardIds.stream().map(trackedLeases::get).filter(Objects::nonNull) - .collect(Collectors.toList()); - - if (leaseForClosedShard != null && leaseForClosedShard.checkpoint().equals(ExtendedSequenceNumber.SHARD_END) - && childShardLeases.size() == childShardIds.size()) { - boolean okayToDelete = true; - for (Lease lease : childShardLeases) { - if (lease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON)) { - okayToDelete = false; - break; - } - } - - if (okayToDelete) { - log.info("Deleting lease for shard {} as it has been completely processed and processing of child " - + "shards has begun.", leaseForClosedShard.leaseKey()); - leaseRefresher.deleteLease(leaseForClosedShard); - } + private static Lease newKCLMultiStreamLeaseForChildShard(final ChildShard childShard, final StreamIdentifier streamIdentifier) throws InvalidStateException { + MultiStreamLease newLease = new MultiStreamLease(); + newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), childShard.shardId())); + if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + newLease.parentShardIds(childShard.parentShards()); + } else { + throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found."); } + newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.streamIdentifier(streamIdentifier.serialize()); + newLease.shardId(childShard.shardId()); + newLease.hashKeyRange(fromHashKeyRange(childShard.hashKeyRange())); + return newLease; } /** @@ -680,7 +625,25 @@ public class HierarchicalShardSyncer { } newLease.parentShardIds(parentShardIds); newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); + return newLease; + } + private static Lease newKCLMultiStreamLease(final Shard shard, final StreamIdentifier streamIdentifier) { + MultiStreamLease newLease = new MultiStreamLease(); + newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), shard.shardId())); + List parentShardIds = new ArrayList<>(2); + if (shard.parentShardId() != null) { + parentShardIds.add(shard.parentShardId()); + } + if (shard.adjacentParentShardId() != null) { + parentShardIds.add(shard.adjacentParentShardId()); + } + newLease.parentShardIds(parentShardIds); + newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.streamIdentifier(streamIdentifier.serialize()); + newLease.shardId(shard.shardId()); + newLease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); return newLease; } @@ -701,9 +664,9 @@ public class HierarchicalShardSyncer { * @param allShards All shards returved via DescribeStream. We assume this to represent a consistent shard list. * @return List of open shards (shards at the tip of the stream) - may include shards that are not yet active. */ - static List getOpenShards(final List allShards) { + static List getOpenShards(final List allShards, final String streamIdentifier) { return allShards.stream().filter(shard -> shard.sequenceNumberRange().endingSequenceNumber() == null) - .peek(shard -> log.debug("Found open shard: {}", shard.shardId())).collect(Collectors.toList()); + .peek(shard -> log.debug("{} : Found open shard: {}", streamIdentifier, shard.shardId())).collect(Collectors.toList()); } private static ExtendedSequenceNumber convertToCheckpoint(final InitialPositionInStreamExtended position) { @@ -719,7 +682,34 @@ public class HierarchicalShardSyncer { return checkpoint; } - + + private static String getStreamIdentifier(MultiStreamArgs multiStreamArgs) { + return Optional.ofNullable(multiStreamArgs.streamIdentifier()) + .map(streamId -> streamId.serialize()).orElse("single_stream_mode"); + } + + /** + * Helper class to compare shards based on their hash range. + */ + @RequiredArgsConstructor + private static class ShardStartingHashKeyBasedComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + + /** + * Compares two shards based on their starting hash keys. + * We assume that the shards provided are non-null. + * + * {@inheritDoc} + */ + @Override + public int compare(Shard shard1, Shard shard2) { + BigInteger hashKey1 = new BigInteger(shard1.hashKeyRange().startingHashKey()); + BigInteger hashKey2 = new BigInteger(shard2.hashKeyRange().startingHashKey()); + + return hashKey1.compareTo(hashKey2); + } + } + /** Helper class to compare leases based on starting sequence number of the corresponding shards. * */ @@ -728,6 +718,7 @@ public class HierarchicalShardSyncer { private static final long serialVersionUID = 1L; private final Map shardIdToShardMap; + private final MultiStreamArgs multiStreamArgs; /** * Compares two leases based on the starting sequence number of corresponding shards. @@ -741,8 +732,8 @@ public class HierarchicalShardSyncer { @Override public int compare(final Lease lease1, final Lease lease2) { int result = 0; - final String shardId1 = lease1.leaseKey(); - final String shardId2 = lease2.leaseKey(); + final String shardId1 = shardIdFromLeaseDeducer.apply(lease1, multiStreamArgs); + final String shardId2 = shardIdFromLeaseDeducer.apply(lease2, multiStreamArgs); final Shard shard1 = shardIdToShardMap.get(shardId1); final Shard shard2 = shardIdToShardMap.get(shardId2); @@ -762,4 +753,243 @@ public class HierarchicalShardSyncer { } + @Data + @Accessors(fluent = true) + @VisibleForTesting + static class MultiStreamArgs { + private final Boolean isMultiStreamMode; + private final StreamIdentifier streamIdentifier; + } + + /** + * Interface to determine how to create new leases. + */ + @VisibleForTesting + interface LeaseSynchronizer { + /** + * Determines how to create leases. + * @param shards + * @param currentLeases + * @param initialPosition + * @param inconsistentShardIds + * @param multiStreamArgs + * @return + */ + List determineNewLeasesToCreate(List shards, List currentLeases, + InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, + MultiStreamArgs multiStreamArgs); + } + + /** + * Class to help create leases when the table is initially empty. + */ + @Slf4j + @AllArgsConstructor + static class EmptyLeaseTableSynchronizer implements LeaseSynchronizer { + + /** + * Determines how to create leases when the lease table is initially empty. For this, we read all shards where + * the KCL is reading from. For any shards which are closed, we will discover their child shards through GetRecords + * child shard information. + * + * @param shards + * @param currentLeases + * @param initialPosition + * @param inconsistentShardIds + * @param multiStreamArgs + * @return + */ + @Override + public List determineNewLeasesToCreate(List shards, List currentLeases, + InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, MultiStreamArgs multiStreamArgs) { + final String streamIdentifier = Optional.ofNullable(multiStreamArgs.streamIdentifier()) + .map(streamId -> streamId.serialize()).orElse(""); + final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); + + currentLeases.stream().peek(lease -> log.debug("{} : Existing lease: {}", streamIdentifier, lease)) + .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .collect(Collectors.toSet()); + + final List newLeasesToCreate = getLeasesToCreateForOpenAndClosedShards(initialPosition, shards, multiStreamArgs, streamIdentifier); + + //TODO: Verify before LTR launch that ending sequence number is still returned from the service. + final Comparator startingSequenceNumberComparator = + new StartingSequenceNumberAndShardIdBasedComparator(shardIdToShardMapOfAllKinesisShards, multiStreamArgs); + newLeasesToCreate.sort(startingSequenceNumberComparator); + return newLeasesToCreate; + } + + /** + * Helper method to create leases. For an empty lease table, we will be creating leases for all shards + * regardless of if they are open or closed. Closed shards will be unblocked via child shard information upon + * reaching SHARD_END. + */ + private List getLeasesToCreateForOpenAndClosedShards(InitialPositionInStreamExtended initialPosition, + List shards, MultiStreamArgs multiStreamArgs, String streamId) { + final Map shardIdToNewLeaseMap = new HashMap<>(); + + for (Shard shard : shards) { + final String shardId = shard.shardId(); + final Lease lease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier) : newKCLLease(shard); + lease.checkpoint(convertToCheckpoint(initialPosition)); + + log.debug("{} : Need to create a lease for shard with shardId {}", streamId, shardId); + + shardIdToNewLeaseMap.put(shardId, lease); + } + + return new ArrayList(shardIdToNewLeaseMap.values()); + } + } + + + /** + * Class to help create leases when the lease table is not initially empty. + */ + @Slf4j + @AllArgsConstructor + static class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { + + private final ShardDetector shardDetector; + private final Map shardIdToShardMap; + private final Map> shardIdToChildShardIdsMap; + + /** + * Determine new leases to create and their initial checkpoint. + * 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 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 + * we begin processing data from any of its descendants. + * * A shard does not start processing data until data from all its parents has been processed. + * Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create + * leases corresponding to both the parents - the parent shard which is not a descendant will have + * its checkpoint set to Latest. + *

+ * We assume that if there is an existing lease for a shard, then either: + * * we have previously created a lease for its parent (if it was needed), or + * * the parent shard has expired. + *

+ * For example: + * 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) + * + * 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 + * before creating all the leases. + *

+ * If a shard has no existing lease, is open, and is a descendant of a parent which is still open, we ignore it + * here; this happens when the list of shards is inconsistent, which could be due to pagination delay for very + * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only + * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. + * + * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard + */ + @Override + public synchronized List determineNewLeasesToCreate(final List shards, final List currentLeases, + final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds, + final MultiStreamArgs multiStreamArgs) { + final Map shardIdToNewLeaseMap = new HashMap<>(); + final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); + final String streamIdentifier = Optional.ofNullable(multiStreamArgs.streamIdentifier()) + .map(streamId -> streamId.serialize()).orElse(""); + final Set shardIdsOfCurrentLeases = currentLeases.stream() + .peek(lease -> log.debug("{} : Existing lease: {}", streamIdentifier, lease)) + .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .collect(Collectors.toSet()); + + final List openShards = getOpenShards(shards, streamIdentifier); + final MemoizationContext memoizationContext = new MemoizationContext(); + + // Iterate over the open shards and find those that don't have any lease entries. + for (Shard shard : openShards) { + final String shardId = shard.shardId(); + log.debug("{} : Evaluating leases for open shard {} and its ancestors.", streamIdentifier, shardId); + if (shardIdsOfCurrentLeases.contains(shardId)) { + log.debug("{} : Lease for shardId {} already exists. Not creating a lease", streamIdentifier, shardId); + } else if (inconsistentShardIds.contains(shardId)) { + log.info("{} : shardId {} is an inconsistent child. Not creating a lease", streamIdentifier, shardId); + } else { + log.debug("{} : Beginning traversal of ancestry tree for shardId {}", streamIdentifier, 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. + final boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, + shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, + memoizationContext, multiStreamArgs); + + // 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 {} has no ancestors. Creating a lease.", streamIdentifier, shardId); + final Lease newLease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : + newKCLLease(shard); + newLease.checkpoint(convertToCheckpoint(initialPosition)); + log.debug("{} : Set checkpoint of {} to {}", streamIdentifier, newLease.leaseKey(), newLease.checkpoint()); + shardIdToNewLeaseMap.put(shardId, newLease); + } else { + log.debug("{} : shardId {} is a descendant whose ancestors should already have leases. " + + "Not creating a lease.", streamIdentifier, shardId); + } + } + } + + final List newLeasesToCreate = new ArrayList<>(shardIdToNewLeaseMap.values()); + final Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( + shardIdToShardMapOfAllKinesisShards, multiStreamArgs); + newLeasesToCreate.sort(startingSequenceNumberComparator); + return newLeasesToCreate; + } + } + + /** + * 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/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java index ba136f0a..96a0de6a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -19,34 +19,41 @@ import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; - -import org.apache.commons.lang3.StringUtils; - import lombok.AccessLevel; import lombok.Getter; import lombok.NonNull; import lombok.Synchronized; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse; import software.amazon.awssdk.services.kinesis.model.KinesisException; import software.amazon.awssdk.services.kinesis.model.LimitExceededException; import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; import software.amazon.kinesis.common.KinesisRequestsBuilder; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.retrieval.AWSExceptionManager; /** @@ -59,8 +66,8 @@ public class KinesisShardDetector implements ShardDetector { @NonNull private final KinesisAsyncClient kinesisClient; - @NonNull - private final String streamName; + @NonNull @Getter + private final StreamIdentifier streamIdentifier; private final long listShardsBackoffTimeInMillis; private final int maxListShardsRetryAttempts; private final long listShardsCacheAllowedAgeInSeconds; @@ -77,16 +84,16 @@ public class KinesisShardDetector implements ShardDetector { public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload, int cacheMissWarningModulus) { - this(kinesisClient, streamName, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, + this(kinesisClient, StreamIdentifier.singleStreamInstance(streamName), listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); } - public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, + public KinesisShardDetector(KinesisAsyncClient kinesisClient, StreamIdentifier streamIdentifier, long listShardsBackoffTimeInMillis, int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload, int cacheMissWarningModulus, Duration kinesisRequestTimeout) { this.kinesisClient = kinesisClient; - this.streamName = streamName; + this.streamIdentifier = streamIdentifier; this.listShardsBackoffTimeInMillis = listShardsBackoffTimeInMillis; this.maxListShardsRetryAttempts = maxListShardsRetryAttempts; this.listShardsCacheAllowedAgeInSeconds = listShardsCacheAllowedAgeInSeconds; @@ -149,12 +156,18 @@ public class KinesisShardDetector implements ShardDetector { @Override @Synchronized public List listShards() { + return listShardsWithFilter(null); + } + + @Override + @Synchronized + public List listShardsWithFilter(ShardFilter shardFilter) { final List shards = new ArrayList<>(); ListShardsResponse result; String nextToken = null; do { - result = listShards(nextToken); + result = listShards(shardFilter, nextToken); if (result == null) { /* @@ -172,15 +185,16 @@ public class KinesisShardDetector implements ShardDetector { return shards; } - private ListShardsResponse listShards(final String nextToken) { + private ListShardsResponse listShards(ShardFilter shardFilter, final String nextToken) { final AWSExceptionManager exceptionManager = new AWSExceptionManager(); + exceptionManager.add(ResourceNotFoundException.class, t -> t); exceptionManager.add(LimitExceededException.class, t -> t); exceptionManager.add(ResourceInUseException.class, t -> t); exceptionManager.add(KinesisException.class, t -> t); - ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder(); + ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder().shardFilter(shardFilter); if (StringUtils.isEmpty(nextToken)) { - request = request.streamName(streamName); + request = request.streamName(streamIdentifier.streamName()); } else { request = request.nextToken(nextToken); } @@ -189,10 +203,9 @@ public class KinesisShardDetector implements ShardDetector { int remainingRetries = maxListShardsRetryAttempts; while (result == null) { - try { try { - result = FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request.build()), kinesisRequestTimeout); + result = getListShardsResponse(request.build()); } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } catch (InterruptedException e) { @@ -205,14 +218,20 @@ public class KinesisShardDetector implements ShardDetector { + " Active or Updating)"); return null; } catch (LimitExceededException e) { - log.info("Got LimitExceededException when listing shards {}. Backing off for {} millis.", streamName, + log.info("Got LimitExceededException when listing shards {}. Backing off for {} millis.", streamIdentifier, listShardsBackoffTimeInMillis); try { Thread.sleep(listShardsBackoffTimeInMillis); } catch (InterruptedException ie) { - log.debug("Stream {} : Sleep was interrupted ", streamName, ie); + log.debug("Stream {} : Sleep was interrupted ", streamIdentifier, ie); } lastException = e; + } catch (ResourceNotFoundException e) { + log.warn("Got ResourceNotFoundException when fetching shard list for {}. Stream no longer exists.", + streamIdentifier.streamName()); + return ListShardsResponse.builder().shards(Collections.emptyList()) + .nextToken(null) + .build(); } catch (TimeoutException te) { throw new RuntimeException(te); } @@ -243,4 +262,31 @@ public class KinesisShardDetector implements ShardDetector { log.debug("{}. Age doesn't exceed limit of {} seconds.", message, listShardsCacheAllowedAgeInSeconds); return false; } + + @Override + public ListShardsResponse getListShardsResponse(ListShardsRequest request) throws + ExecutionException, TimeoutException, InterruptedException { + return FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request), kinesisRequestTimeout); + } + + @Override + public List getChildShards(final String shardId) throws InterruptedException, ExecutionException, TimeoutException { + final GetShardIteratorRequest getShardIteratorRequest = KinesisRequestsBuilder.getShardIteratorRequestBuilder() + .streamName(streamIdentifier.streamName()) + .shardIteratorType(ShardIteratorType.LATEST) + .shardId(shardId) + .build(); + + final GetShardIteratorResponse getShardIteratorResponse = + FutureUtils.resolveOrCancelFuture(kinesisClient.getShardIterator(getShardIteratorRequest), kinesisRequestTimeout); + + final GetRecordsRequest getRecordsRequest = KinesisRequestsBuilder.getRecordsRequestBuilder() + .shardIterator(getShardIteratorResponse.shardIterator()) + .build(); + + final GetRecordsResponse getRecordsResponse = + FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(getRecordsRequest), kinesisRequestTimeout); + + return getRecordsResponse.childShards(); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java index 802ee29b..359b7a44 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java @@ -14,22 +14,21 @@ */ package software.amazon.kinesis.leases; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; - -import com.google.common.collect.Collections2; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.ToString; import lombok.experimental.Accessors; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + /** * This class contains data pertaining to a Lease. Distributed systems may use leases to partition work across a * fleet of workers. Each unit of work (identified by a leaseKey) has a corresponding Lease. Every worker will contend @@ -40,7 +39,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @NoArgsConstructor @Getter @Accessors(fluent = true) -@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos"}) +@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "childShardIds", "pendingCheckpointState"}) @ToString public class Lease { /* @@ -84,11 +83,21 @@ public class Lease { * @return pending checkpoint, possibly null. */ private ExtendedSequenceNumber pendingCheckpoint; + + /** + * Last pending application state. Deliberately excluded from hashCode and equals. + * + * @return pending checkpoint state, possibly null. + */ + private byte[] pendingCheckpointState; + /** * @return count of distinct lease holders between checkpoints. */ private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); + private HashKeyRangeForLease hashKeyRangeForLease; /** * Copy constructor, used by clone(). @@ -98,13 +107,24 @@ public class Lease { protected Lease(Lease lease) { this(lease.leaseKey(), lease.leaseOwner(), lease.leaseCounter(), lease.concurrencyToken(), lease.lastCounterIncrementNanos(), lease.checkpoint(), lease.pendingCheckpoint(), - lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds()); + lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.childShardIds(), + lease.pendingCheckpointState(), lease.hashKeyRangeForLease()); + } + + @Deprecated + public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, + final UUID concurrencyToken, final Long lastCounterIncrementNanos, + final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, + final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds) { + this(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, pendingCheckpoint, + ownerSwitchesSinceCheckpoint, parentShardIds, new HashSet<>(), null, null); } public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, final UUID concurrencyToken, final Long lastCounterIncrementNanos, final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, - final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds) { + final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds, final Set childShardIds, + final byte[] pendingCheckpointState, final HashKeyRangeForLease hashKeyRangeForLease) { this.leaseKey = leaseKey; this.leaseOwner = leaseOwner; this.leaseCounter = leaseCounter; @@ -116,6 +136,11 @@ public class Lease { if (parentShardIds != null) { this.parentShardIds.addAll(parentShardIds); } + if (childShardIds != null) { + this.childShardIds.addAll(childShardIds); + } + this.hashKeyRangeForLease = hashKeyRangeForLease; + this.pendingCheckpointState = pendingCheckpointState; } /** @@ -135,7 +160,9 @@ public class Lease { ownerSwitchesSinceCheckpoint(lease.ownerSwitchesSinceCheckpoint()); checkpoint(lease.checkpoint); pendingCheckpoint(lease.pendingCheckpoint); + pendingCheckpointState(lease.pendingCheckpointState); parentShardIds(lease.parentShardIds); + childShardIds(lease.childShardIds); } /** @@ -214,6 +241,15 @@ public class Lease { this.pendingCheckpoint = pendingCheckpoint; } + /** + * Sets pending checkpoint state. + * + * @param pendingCheckpointState can be null + */ + public void pendingCheckpointState(byte[] pendingCheckpointState) { + this.pendingCheckpointState = pendingCheckpointState; + } + /** * Sets ownerSwitchesSinceCheckpoint. * @@ -233,6 +269,27 @@ public class Lease { this.parentShardIds.addAll(parentShardIds); } + /** + * Sets childShardIds. + * + * @param childShardIds may not be null + */ + public void childShardIds(@NonNull final Collection childShardIds) { + this.childShardIds.addAll(childShardIds); + } + + /** + * Set the hash range key for this shard. + * @param hashKeyRangeForLease + */ + public void hashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) { + if (this.hashKeyRangeForLease == null) { + this.hashKeyRangeForLease = hashKeyRangeForLease; + } else if (!this.hashKeyRangeForLease.equals(hashKeyRangeForLease)) { + throw new IllegalArgumentException("hashKeyRange is immutable"); + } + } + /** * Sets leaseOwner. * @@ -250,4 +307,6 @@ public class Lease { public Lease copy() { return new Lease(this); } + + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java new file mode 100644 index 00000000..6e3104ae --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java @@ -0,0 +1,348 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.Value; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; +import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.retrieval.AWSExceptionManager; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * Helper class to cleanup of any expired/closed shard leases. It will cleanup leases periodically as defined by + * {@link LeaseManagementConfig#leaseCleanupConfig()} asynchronously. + */ +@Accessors(fluent=true) +@Slf4j +@RequiredArgsConstructor +@EqualsAndHashCode +public class LeaseCleanupManager { + @NonNull + private final LeaseCoordinator leaseCoordinator; + @NonNull + private final MetricsFactory metricsFactory; + @NonNull + private final ScheduledExecutorService deletionThreadPool; + private final boolean cleanupLeasesUponShardCompletion; + private final long leaseCleanupIntervalMillis; + private final long completedLeaseCleanupIntervalMillis; + private final long garbageLeaseCleanupIntervalMillis; + private final Stopwatch completedLeaseStopwatch = Stopwatch.createUnstarted(); + private final Stopwatch garbageLeaseStopwatch = Stopwatch.createUnstarted(); + + private final Queue deletionQueue = new ConcurrentLinkedQueue<>(); + + private static final long INITIAL_DELAY = 0L; + + @Getter + private volatile boolean isRunning = false; + + /** + * Starts the lease cleanup thread, which is scheduled periodically as specified by + * {@link LeaseCleanupManager#leaseCleanupIntervalMillis} + */ + public void start() { + if (!isRunning) { + log.info("Starting lease cleanup thread."); + completedLeaseStopwatch.reset().start(); + garbageLeaseStopwatch.reset().start(); + deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, + TimeUnit.MILLISECONDS); + isRunning = true; + } else { + log.info("Lease cleanup thread already running, no need to start."); + } + } + + /** + * Enqueues a lease for deletion without check for duplicate entry. Use {@link #isEnqueuedForDeletion} + * for checking the duplicate entries. + * @param leasePendingDeletion + */ + public void enqueueForDeletion(LeasePendingDeletion leasePendingDeletion) { + final Lease lease = leasePendingDeletion.lease(); + if (lease == null) { + log.warn("Cannot enqueue lease {} for deferred deletion - instance doesn't hold the lease for that shard.", + lease.leaseKey()); + } else { + log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey()); + if (!deletionQueue.add(leasePendingDeletion)) { + log.warn("Unable to enqueue lease {} for deletion.", lease.leaseKey()); + } + } + } + + /** + * Check if lease was already enqueued for deletion. + * //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597. + * @param leasePendingDeletion + * @return true if enqueued for deletion; false otherwise. + */ + public boolean isEnqueuedForDeletion(LeasePendingDeletion leasePendingDeletion) { + return deletionQueue.contains(leasePendingDeletion); + } + + /** + * Returns how many leases are currently waiting in the queue pending deletion. + * @return number of leases pending deletion. + */ + private int leasesPendingDeletion() { + return deletionQueue.size(); + } + + private boolean timeToCheckForCompletedShard() { + return completedLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= completedLeaseCleanupIntervalMillis; + } + + private boolean timeToCheckForGarbageShard() { + return garbageLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= garbageLeaseCleanupIntervalMillis; + } + + public LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion, + boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException, + InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { + final Lease lease = leasePendingDeletion.lease(); + final ShardInfo shardInfo = leasePendingDeletion.shardInfo(); + final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier(); + + final AWSExceptionManager exceptionManager = createExceptionManager(); + + boolean cleanedUpCompletedLease = false; + boolean cleanedUpGarbageLease = false; + boolean alreadyCheckedForGarbageCollection = false; + boolean wereChildShardsPresent = false; + boolean wasResourceNotFound = false; + + try { + if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) { + final Lease leaseFromDDB = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey()); + if(leaseFromDDB != null) { + Set childShardKeys = leaseFromDDB.childShardIds(); + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + try { + childShardKeys = leasePendingDeletion.getChildShardsFromService(); + + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + log.error( + "No child shards returned from service for shard {} for {} while cleaning up lease.", + shardInfo.shardId(), streamIdentifier.streamName()); + } else { + wereChildShardsPresent = true; + updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + } + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } finally { + alreadyCheckedForGarbageCollection = true; + } + } else { + wereChildShardsPresent = true; + } + try { + cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys); + } catch (Exception e) { + // Suppressing the exception here, so that we can attempt for garbage cleanup. + log.warn("Unable to cleanup lease for shard {} in {}", shardInfo.shardId(), streamIdentifier.streamName(), e); + } + } else { + log.info("Lease not present in lease table while cleaning the shard {} of {}", shardInfo.shardId(), streamIdentifier.streamName()); + cleanedUpCompletedLease = true; + } + } + + if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) { + try { + wereChildShardsPresent = !CollectionUtils + .isNullOrEmpty(leasePendingDeletion.getChildShardsFromService()); + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } + } + } catch (ResourceNotFoundException e) { + wasResourceNotFound = true; + cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease, e); + } + + return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease, wereChildShardsPresent, + wasResourceNotFound); + } + + // A lease that ended with SHARD_END from ResourceNotFoundException is safe to delete if it no longer exists in the + // stream (known explicitly from ResourceNotFound being thrown when processing this shard), + private boolean cleanupLeaseForGarbageShard(Lease lease, Throwable e) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + log.warn("Deleting lease {} as it is not present in the stream.", lease, e); + leaseCoordinator.leaseRefresher().deleteLease(lease); + return true; + } + + private boolean allParentShardLeasesDeleted(Lease lease, ShardInfo shardInfo) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + for (String parentShard : lease.parentShardIds()) { + final Lease parentLease = leaseCoordinator.leaseRefresher().getLease(ShardInfo.getLeaseKey(shardInfo, parentShard)); + + if (parentLease != null) { + log.warn("Lease {} has a parent lease {} which is still present in the lease table, skipping deletion " + + "for this lease.", lease, parentLease); + return false; + } + } + return true; + } + + // We should only be deleting the current shard's lease if + // 1. All of its children are currently being processed, i.e their checkpoint is not TRIM_HORIZON or AT_TIMESTAMP. + // 2. Its parent shard lease(s) have already been deleted. + private boolean cleanupLeaseForCompletedShard(Lease lease, ShardInfo shardInfo, Set childShardKeys) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, IllegalStateException { + final Set processedChildShardLeaseKeys = new HashSet<>(); + final Set childShardLeaseKeys = childShardKeys.stream().map(ck -> ShardInfo.getLeaseKey(shardInfo, ck)) + .collect(Collectors.toSet()); + + for (String childShardLeaseKey : childShardLeaseKeys) { + final Lease childShardLease = Optional.ofNullable( + leaseCoordinator.leaseRefresher().getLease(childShardLeaseKey)) + .orElseThrow(() -> new IllegalStateException( + "Child lease " + childShardLeaseKey + " for completed shard not found in " + + "lease table - not cleaning up lease " + lease)); + + if (!childShardLease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) && !childShardLease + .checkpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) { + processedChildShardLeaseKeys.add(childShardLease.leaseKey()); + } + } + + if (!allParentShardLeasesDeleted(lease, shardInfo) || !Objects.equals(childShardLeaseKeys, processedChildShardLeaseKeys)) { + return false; + } + + log.info("Deleting lease {} as it has been completely processed and processing of child shard(s) has begun.", + lease); + leaseCoordinator.leaseRefresher().deleteLease(lease); + + return true; + } + + private void updateLeaseWithChildShards(LeasePendingDeletion leasePendingDeletion, Set childShardKeys) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final Lease updatedLease = leasePendingDeletion.lease(); + updatedLease.childShardIds(childShardKeys); + + leaseCoordinator.leaseRefresher().updateLeaseWithMetaInfo(updatedLease, UpdateField.CHILD_SHARDS); + } + + private AWSExceptionManager createExceptionManager() { + final AWSExceptionManager exceptionManager = new AWSExceptionManager(); + exceptionManager.add(ResourceNotFoundException.class, t -> t); + + return exceptionManager; + } + + @VisibleForTesting + void cleanupLeases() { + log.info("Number of pending leases to clean before the scan : {}", leasesPendingDeletion()); + if (deletionQueue.isEmpty()) { + log.debug("No leases pending deletion."); + } else if (timeToCheckForCompletedShard() | timeToCheckForGarbageShard()) { + final Queue failedDeletions = new ConcurrentLinkedQueue<>(); + boolean completedLeaseCleanedUp = false; + boolean garbageLeaseCleanedUp = false; + + log.debug("Attempting to clean up {} lease(s).", deletionQueue.size()); + + while (!deletionQueue.isEmpty()) { + final LeasePendingDeletion leasePendingDeletion = deletionQueue.poll(); + final String leaseKey = leasePendingDeletion.lease().leaseKey(); + final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier(); + boolean deletionSucceeded = false; + try { + final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion, + timeToCheckForCompletedShard(), timeToCheckForGarbageShard()); + completedLeaseCleanedUp |= leaseCleanupResult.cleanedUpCompletedLease(); + garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease(); + + if (leaseCleanupResult.leaseCleanedUp()) { + log.info("Successfully cleaned up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult); + deletionSucceeded = true; + } else { + log.warn("Unable to clean up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult); + } + } catch (Exception e) { + log.error("Failed to cleanup lease {} for {}. Will re-enqueue for deletion and retry on next " + + "scheduled execution.", leaseKey, streamIdentifier, e); + } + if (!deletionSucceeded) { + log.debug("Did not cleanup lease {} for {}. Re-enqueueing for deletion.", leaseKey, streamIdentifier); + failedDeletions.add(leasePendingDeletion); + } + } + if (completedLeaseCleanedUp) { + log.debug("At least one completed lease was cleaned up - restarting interval"); + completedLeaseStopwatch.reset().start(); + } + if (garbageLeaseCleanedUp) { + log.debug("At least one garbage lease was cleaned up - restarting interval"); + garbageLeaseStopwatch.reset().start(); + } + deletionQueue.addAll(failedDeletions); + + log.info("Number of pending leases to clean after the scan : {}", leasesPendingDeletion()); + } + } + + private class LeaseCleanupThread implements Runnable { + @Override + public void run() { + cleanupLeases(); + } + } + + @Value + public static class LeaseCleanupResult { + boolean cleanedUpCompletedLease; + boolean cleanedUpGarbageLease; + boolean wereChildShardsPresent; + boolean wasResourceNotFound; + + public boolean leaseCleanedUp() { + return cleanedUpCompletedLease | cleanedUpGarbageLease; + } + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java index 880fab4c..6437f339 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java @@ -92,6 +92,9 @@ public interface LeaseCoordinator { * * @param lease lease object containing updated values * @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease + * @param operation that performs updateLease + * @param singleStreamShardId for metrics emission in single stream mode. MultiStream mode will get the + * shardId from the lease object * * @return true if update succeeded, false otherwise * @@ -99,7 +102,7 @@ public interface LeaseCoordinator { * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws DependencyException if DynamoDB update fails in an unexpected way */ - boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String shardId) + boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException; /** diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java index 20e0aa8f..473db5bb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java @@ -15,23 +15,26 @@ package software.amazon.kinesis.leases; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + import java.time.Duration; import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; - +import java.util.function.Function; import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; +import org.apache.commons.lang3.Validate; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.LeaseCleanupConfig; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseManagementFactory; import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback; import software.amazon.kinesis.metrics.MetricsFactory; @@ -46,6 +49,19 @@ public class LeaseManagementConfig { public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofMinutes(1); + public static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(1).toMillis(); + public static final long DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(5).toMillis(); + public static final long DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(30).toMillis(); + public static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L; + public static final int DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY = 3; + + + public static final LeaseCleanupConfig DEFAULT_LEASE_CLEANUP_CONFIG = LeaseCleanupConfig.builder() + .leaseCleanupIntervalMillis(DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS) + .completedLeaseCleanupIntervalMillis(DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS) + .garbageLeaseCleanupIntervalMillis(DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS) + .build(); + /** * Name of the table to use in DynamoDB * @@ -71,7 +87,7 @@ public class LeaseManagementConfig { * Name of the Kinesis Data Stream to read records from. */ @NonNull - private final String streamName; + private String streamName; /** * Used to distinguish different workers/processes of a KCL application. * @@ -106,6 +122,15 @@ public class LeaseManagementConfig { */ private boolean cleanupLeasesUponShardCompletion = true; + /** + * Configuration for lease cleanup in {@link LeaseCleanupManager}. + * + *

Default lease cleanup interval value: 1 minute.

+ *

Default completed lease cleanup threshold: 5 minute.

+ *

Default garbage lease cleanup threshold: 30 minute.

+ */ + private final LeaseCleanupConfig leaseCleanupConfig = DEFAULT_LEASE_CLEANUP_CONFIG; + /** * The max number of leases (shards) this worker should process. * This can be useful to avoid overloading (and thrashing) a worker when a host has resource constraints @@ -116,7 +141,7 @@ public class LeaseManagementConfig { * *

Default value: {@link Integer#MAX_VALUE}

*/ - private int maxLeasesForWorker = Integer.MAX_VALUE;; + private int maxLeasesForWorker = Integer.MAX_VALUE; /** * Max leases to steal from another worker at one time (for load balancing). @@ -141,6 +166,11 @@ public class LeaseManagementConfig { */ private int initialLeaseTableWriteCapacity = 10; + /** + * Configurable functional interface to override the existing shardDetector. + */ + private Function customShardDetectorProvider; + /** * The size of the thread pool to create for the lease renewer to use. * @@ -168,6 +198,20 @@ public class LeaseManagementConfig { private BillingMode billingMode = BillingMode.PROVISIONED; + /** + * Frequency (in millis) of the auditor job to scan for partial leases in the lease table. + * If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on + * {@link #leasesRecoveryAuditorInconsistencyConfidenceThreshold} + */ + private long leasesRecoveryAuditorExecutionFrequencyMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS; + + /** + * Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table + * is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, + * then it would trigger a shard sync. + */ + private int leasesRecoveryAuditorInconsistencyConfidenceThreshold = DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY; + /** * The initial position for getting records from Kinesis streams. * @@ -182,6 +226,24 @@ public class LeaseManagementConfig { private MetricsFactory metricsFactory = new NullMetricsFactory(); + @Deprecated + public LeaseManagementConfig(String tableName, DynamoDbAsyncClient dynamoDBClient, KinesisAsyncClient kinesisClient, + String streamName, String workerIdentifier) { + this.tableName = tableName; + this.dynamoDBClient = dynamoDBClient; + this.kinesisClient = kinesisClient; + this.streamName = streamName; + this.workerIdentifier = workerIdentifier; + } + + public LeaseManagementConfig(String tableName, DynamoDbAsyncClient dynamoDBClient, KinesisAsyncClient kinesisClient, + String workerIdentifier) { + this.tableName = tableName; + this.dynamoDBClient = dynamoDBClient; + this.kinesisClient = kinesisClient; + this.workerIdentifier = workerIdentifier; + } + /** * Returns the metrics factory. * @@ -240,12 +302,21 @@ public class LeaseManagementConfig { */ private TableCreatorCallback tableCreatorCallback = TableCreatorCallback.NOOP_TABLE_CREATOR_CALLBACK; - private HierarchicalShardSyncer hierarchicalShardSyncer = new HierarchicalShardSyncer(); + private HierarchicalShardSyncer hierarchicalShardSyncer; private LeaseManagementFactory leaseManagementFactory; + private HierarchicalShardSyncer hierarchicalShardSyncer() { + if(hierarchicalShardSyncer == null) { + hierarchicalShardSyncer = new HierarchicalShardSyncer(); + } + return hierarchicalShardSyncer; + } + + @Deprecated public LeaseManagementFactory leaseManagementFactory() { if (leaseManagementFactory == null) { + Validate.notEmpty(streamName(), "Stream name is empty"); leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), streamName(), dynamoDBClient(), @@ -275,4 +346,54 @@ public class LeaseManagementConfig { return leaseManagementFactory; } + /** + * Vends LeaseManagementFactory that performs serde based on leaseSerializer and shard sync based on isMultiStreamingMode + * @param leaseSerializer + * @param isMultiStreamingMode + * @return LeaseManagementFactory + */ + public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer, boolean isMultiStreamingMode) { + if(leaseManagementFactory == null) { + leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), + dynamoDBClient(), + tableName(), + workerIdentifier(), + executorService(), + failoverTimeMillis(), + epsilonMillis(), + maxLeasesForWorker(), + maxLeasesToStealAtOneTime(), + maxLeaseRenewalThreads(), + cleanupLeasesUponShardCompletion(), + ignoreUnexpectedChildShards(), + shardSyncIntervalMillis(), + consistentReads(), + listShardsBackoffTimeInMillis(), + maxListShardsRetryAttempts(), + maxCacheMissesBeforeReload(), + listShardsCacheAllowedAgeInSeconds(), + cacheMissWarningModulus(), + initialLeaseTableReadCapacity(), + initialLeaseTableWriteCapacity(), + hierarchicalShardSyncer(), + tableCreatorCallback(), + dynamoDbRequestTimeout(), + billingMode(), + leaseSerializer, + customShardDetectorProvider(), + isMultiStreamingMode, + leaseCleanupConfig()); + } + return leaseManagementFactory; + } + + /** + * Set leaseManagementFactory and return the current LeaseManagementConfig instance. + * @param leaseManagementFactory + * @return LeaseManagementConfig + */ + public LeaseManagementConfig leaseManagementFactory(final LeaseManagementFactory leaseManagementFactory) { + this.leaseManagementFactory = leaseManagementFactory; + return this; + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java index 72f48fea..ecf9b390 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java @@ -15,6 +15,7 @@ package software.amazon.kinesis.leases; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.metrics.MetricsFactory; @@ -26,7 +27,18 @@ public interface LeaseManagementFactory { ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory); + default ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { + throw new UnsupportedOperationException(); + } + DynamoDBLeaseRefresher createLeaseRefresher(); ShardDetector createShardDetector(); + + default ShardDetector createShardDetector(StreamConfig streamConfig) { + throw new UnsupportedOperationException(); + } + + LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory); + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java index 3ba22c2b..b7f38a4e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.leases; import java.util.List; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -60,6 +61,18 @@ public interface LeaseRefresher { */ boolean waitUntilLeaseTableExists(long secondsBetweenPolls, long timeoutSeconds) throws DependencyException; + /** + * List all leases for a given stream synchronously. + * + * @throws DependencyException if DynamoDB scan fails in an unexpected way + * @throws InvalidStateException if lease table does not exist + * @throws ProvisionedThroughputException if DynamoDB scan fails due to lack of capacity + * + * @return list of leases + */ + List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, + ProvisionedThroughputException; + /** * List all objects in table synchronously. * @@ -86,15 +99,15 @@ public interface LeaseRefresher { throws DependencyException, InvalidStateException, ProvisionedThroughputException; /** - * @param shardId Get the lease for this shardId + * @param leaseKey Get the lease for this leasekey * * @throws InvalidStateException if lease table does not exist * @throws ProvisionedThroughputException if DynamoDB get fails due to lack of capacity * @throws DependencyException if DynamoDB get fails in an unexpected way * - * @return lease for the specified shardId, or null if one doesn't exist + * @return lease for the specified leaseKey, or null if one doesn't exist */ - Lease getLease(String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException; + Lease getLease(String leaseKey) throws DependencyException, InvalidStateException, ProvisionedThroughputException; /** * Renew a lease by incrementing the lease counter. Conditional on the leaseCounter in DynamoDB matching the leaseCounter @@ -178,6 +191,21 @@ public interface LeaseRefresher { boolean updateLease(Lease lease) throws DependencyException, InvalidStateException, ProvisionedThroughputException; + /** + * Update application-specific fields of the given lease in DynamoDB. Does not update fields managed by the leasing + * library such as leaseCounter, leaseOwner, or leaseKey. + * + * @return true if update succeeded, false otherwise + * + * @throws InvalidStateException if lease table does not exist + * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity + * @throws DependencyException if DynamoDB update fails in an unexpected way + */ + default void updateLeaseWithMetaInfo(Lease lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throw new UnsupportedOperationException("updateLeaseWithNoExpectation is not implemented"); + } + /** * Check (synchronously) if there are any leases in the lease table. * @@ -193,13 +221,13 @@ public interface LeaseRefresher { * Gets the current checkpoint of the shard. This is useful in the resharding use case * where we will wait for the parent shard to complete before starting on the records from a child shard. * - * @param shardId Checkpoint of this shard will be returned + * @param leaseKey Checkpoint of this shard will be returned * @return Checkpoint of this shard, or null if the shard record doesn't exist. * * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws InvalidStateException if lease table does not exist * @throws DependencyException if DynamoDB update fails in an unexpected way */ - ExtendedSequenceNumber getCheckpoint(String shardId) + ExtendedSequenceNumber getCheckpoint(String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java index 9ed5616f..25ec5b45 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java @@ -86,6 +86,9 @@ public interface LeaseRenewer { * * @param lease lease object containing updated data * @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease + * @param operation that performs updateLease + * @param singleStreamShardId shardId for metrics emission in single stream mode. MultiStream mode will get the + * shardId from the lease object * * @return true if update succeeds, false otherwise * @@ -93,7 +96,7 @@ public interface LeaseRenewer { * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws DependencyException if DynamoDB update fails in an unexpected way */ - boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String shardId) + boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java index b8aa0339..f36f5a66 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java @@ -16,14 +16,11 @@ package software.amazon.kinesis.leases; import java.util.Collection; import java.util.Map; - - import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue; import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; -import software.amazon.kinesis.leases.Lease; /** * Utility class that manages the mapping of Lease objects/operations to records in DynamoDB. @@ -46,6 +43,11 @@ public interface LeaseSerializer { */ Lease fromDynamoRecord(Map dynamoRecord); + + default Lease fromDynamoRecord(Map dynamoRecord, Lease leaseToUpdate) { + throw new UnsupportedOperationException(); + } + /** * @param lease * @return the attribute value map representing a Lease's hash key given a Lease object. @@ -77,6 +79,14 @@ public interface LeaseSerializer { */ Map getDynamoNonexistantExpectation(); + /** + * @param leaseKey + * @return the attribute value map asserting that a lease does exist. + */ + default Map getDynamoExistentExpectation(String leaseKey) { + throw new UnsupportedOperationException("DynamoExistantExpectation is not implemented"); + } + /** * @param lease * @return the attribute value map that increments a lease counter @@ -103,6 +113,15 @@ public interface LeaseSerializer { */ Map getDynamoUpdateLeaseUpdate(Lease lease); + /** + * @param lease + * @param updateField + * @return the attribute value map that updates application-specific data for a lease + */ + default Map getDynamoUpdateLeaseUpdate(Lease lease, UpdateField updateField) { + throw new UnsupportedOperationException(); + } + /** * @return the key schema for creating a DynamoDB table to store leases */ @@ -112,4 +131,5 @@ public interface LeaseSerializer { * @return attribute definitions for creating a DynamoDB table to store leases */ Collection getAttributeDefinitions(); + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java new file mode 100644 index 00000000..c8811354 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java @@ -0,0 +1,80 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.commons.lang3.Validate; + +import java.util.Objects; + +import static com.google.common.base.Verify.verifyNotNull; + +@Setter +@NoArgsConstructor +@Getter +@Accessors(fluent = true) +@EqualsAndHashCode(callSuper = true) +public class MultiStreamLease extends Lease { + + @NonNull private String streamIdentifier; + @NonNull private String shardId; + + public MultiStreamLease(MultiStreamLease other) { + super(other); + streamIdentifier(other.streamIdentifier); + shardId(other.shardId); + } + + @Override + public void update(Lease other) { + MultiStreamLease casted = validateAndCast(other); + super.update(casted); + streamIdentifier(casted.streamIdentifier); + shardId(casted.shardId); + } + + public static String getLeaseKey(String streamIdentifier, String shardId) { + verifyNotNull(streamIdentifier, "streamIdentifier should not be null"); + verifyNotNull(shardId, "shardId should not be null"); + return streamIdentifier + ":" + shardId; + } + + /** + * Returns a deep copy of this object. Type-unsafe - there aren't good mechanisms for copy-constructing generics. + * + * @return A deep copy of this object. + */ + @Override + public MultiStreamLease copy() { + return new MultiStreamLease(this); + } + + /** + * Validate and cast the lease to MultiStream lease + * @param lease + * @return MultiStreamLease + */ + public static MultiStreamLease validateAndCast(Lease lease) { + Validate.isInstanceOf(MultiStreamLease.class, lease); + return (MultiStreamLease) lease; + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java index cf3a1a78..62b93855 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java @@ -15,16 +15,75 @@ package software.amazon.kinesis.leases; -import software.amazon.awssdk.services.kinesis.model.Shard; - import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; +import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.kinesis.common.StreamIdentifier; /** * */ public interface ShardDetector { + + /** + * Gets shard based on shardId. + * + * @param shardId + * @return Shard + */ Shard shard(String shardId); + /** + * List shards. + * + * @return Shards + */ List listShards(); + /** + * List shards with shard filter. + * + * @param ShardFilter + * @return Shards + */ + default List listShardsWithFilter(ShardFilter shardFilter) { + throw new UnsupportedOperationException("listShardsWithFilter not available."); + } + + /** + * Gets stream identifier. + * + * @return StreamIdentifier + */ + default StreamIdentifier streamIdentifier() { + throw new UnsupportedOperationException("StreamName not available"); + } + + /** + * Gets a list shards response based on the request. + * + * @param request list shards request + * @return ListShardsResponse which contains list shards response + */ + default ListShardsResponse getListShardsResponse(ListShardsRequest request) throws Exception { + throw new UnsupportedOperationException("getListShardsResponse not available."); + } + + /** + * Gets the children shards of a shard. + * @param shardId + * @return + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + default List getChildShards(String shardId) throws InterruptedException, ExecutionException, TimeoutException { + throw new UnsupportedOperationException("getChildShards not available."); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 89b8f94a..aff3f6f0 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -18,6 +18,7 @@ import java.util.Collection; import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.Optional; import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.builder.HashCodeBuilder; @@ -36,6 +37,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @ToString public class ShardInfo { + private final Optional streamIdentifierSerOpt; private final String shardId; private final String concurrencyToken; // Sorted list of parent shardIds. @@ -54,11 +56,27 @@ public class ShardInfo { * @param checkpoint * the latest checkpoint from lease */ - // TODO: check what values can be null public ShardInfo(@NonNull final String shardId, final String concurrencyToken, final Collection parentShardIds, final ExtendedSequenceNumber checkpoint) { + this(shardId, concurrencyToken, parentShardIds, checkpoint, null); + } + + /** + * Creates a new ShardInfo object that has an option to pass a serialized streamIdentifier. + * The checkpoint is not part of the equality, but is used for debugging output. + * @param shardId + * @param concurrencyToken + * @param parentShardIds + * @param checkpoint + * @param streamIdentifierSer + */ + public ShardInfo(@NonNull final String shardId, + final String concurrencyToken, + final Collection parentShardIds, + final ExtendedSequenceNumber checkpoint, + final String streamIdentifierSer) { this.shardId = shardId; this.concurrencyToken = concurrencyToken; this.parentShardIds = new LinkedList<>(); @@ -69,6 +87,7 @@ public class ShardInfo { // This makes it easy to check for equality in ShardInfo.equals method. Collections.sort(this.parentShardIds); this.checkpoint = checkpoint; + this.streamIdentifierSerOpt = Optional.ofNullable(streamIdentifierSer); } /** @@ -94,7 +113,8 @@ public class ShardInfo { */ @Override public int hashCode() { - return new HashCodeBuilder().append(concurrencyToken).append(parentShardIds).append(shardId).toHashCode(); + return new HashCodeBuilder() + .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamIdentifierSerOpt.orElse("")).toHashCode(); } /** @@ -118,8 +138,30 @@ public class ShardInfo { } ShardInfo other = (ShardInfo) obj; return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken) - .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId).isEquals(); + .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId) + .append(streamIdentifierSerOpt.orElse(""), other.streamIdentifierSerOpt.orElse("")).isEquals(); } + /** + * Utility method to derive lease key from ShardInfo. + * @param shardInfo + * @return lease key + */ + public static String getLeaseKey(ShardInfo shardInfo) { + return getLeaseKey(shardInfo, shardInfo.shardId()); + } + + /** + * Utility method to derive lease key from ShardInfo and shardId to override. + * @param shardInfo + * @param shardIdOverride + * @return lease key + */ + public static String getLeaseKey(ShardInfo shardInfo, String shardIdOverride) { + return shardInfo.streamIdentifierSerOpt().isPresent() ? + MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardIdOverride) : + shardIdOverride; + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java index c59608b2..dd576114 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java @@ -23,6 +23,7 @@ import software.amazon.kinesis.lifecycle.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.lifecycle.TaskType; import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.MetricsLevel; import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; @@ -45,6 +46,7 @@ public class ShardSyncTask implements ConsumerTask { @NonNull private final InitialPositionInStreamExtended initialPosition; private final boolean cleanupLeasesUponShardCompletion; + private final boolean garbageCollectLeases; private final boolean ignoreUnexpectedChildShards; private final long shardSyncTaskIdleTimeMillis; @NonNull @@ -62,17 +64,25 @@ public class ShardSyncTask implements ConsumerTask { public TaskResult call() { Exception exception = null; final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHARD_SYNC_TASK_OPERATION); + boolean shardSyncSuccess = true; try { - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, scope); - if (shardSyncTaskIdleTimeMillis > 0) { + boolean didPerformShardSync = hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, + initialPosition, scope, ignoreUnexpectedChildShards, + leaseRefresher.isLeaseTableEmpty()); + + if (didPerformShardSync && shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); } } catch (Exception e) { log.error("Caught exception while sync'ing Kinesis shards and leases", e); exception = e; + shardSyncSuccess = false; } finally { + // NOTE: This metric is reflecting if a shard sync task succeeds. Customer can use this metric to monitor if + // their application encounter any shard sync failures. This metric can help to detect potential shard stuck issues + // that are due to shard sync failures. + MetricsUtil.addSuccess(scope, "SyncShards", shardSyncSuccess, MetricsLevel.DETAILED); MetricsUtil.endScope(scope); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java index f6db72e3..e03046a0 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java @@ -14,9 +14,11 @@ */ package software.amazon.kinesis.leases; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -24,6 +26,7 @@ import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import software.amazon.kinesis.coordinator.ExecutorStateEvent; import software.amazon.kinesis.lifecycle.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.MetricsFactory; @@ -45,6 +48,7 @@ public class ShardSyncTaskManager { @NonNull private final InitialPositionInStreamExtended initialPositionInStream; private final boolean cleanupLeasesUponShardCompletion; + private final boolean garbageCollectLeases; private final boolean ignoreUnexpectedChildShards; private final long shardSyncIdleTimeMillis; @NonNull @@ -53,6 +57,10 @@ public class ShardSyncTaskManager { private final HierarchicalShardSyncer hierarchicalShardSyncer; @NonNull private final MetricsFactory metricsFactory; + private ConsumerTask currentTask; + private CompletableFuture future; + private AtomicBoolean shardSyncRequestPending; + private final ReentrantLock lock; /** * Constructor. @@ -77,11 +85,14 @@ public class ShardSyncTaskManager { this.leaseRefresher = leaseRefresher; this.initialPositionInStream = initialPositionInStream; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; + this.garbageCollectLeases = true; this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis; this.executorService = executorService; this.hierarchicalShardSyncer = new HierarchicalShardSyncer(); this.metricsFactory = metricsFactory; + this.shardSyncRequestPending = new AtomicBoolean(false); + this.lock = new ReentrantLock(); } /** @@ -105,21 +116,48 @@ public class ShardSyncTaskManager { this.leaseRefresher = leaseRefresher; this.initialPositionInStream = initialPositionInStream; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; + this.garbageCollectLeases = true; this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis; this.executorService = executorService; this.hierarchicalShardSyncer = hierarchicalShardSyncer; this.metricsFactory = metricsFactory; + this.shardSyncRequestPending = new AtomicBoolean(false); + this.lock = new ReentrantLock(); } - private ConsumerTask currentTask; - private Future future; - - public synchronized boolean syncShardAndLeaseInfo() { - return checkAndSubmitNextTask(); + /** + * Call a ShardSyncTask and return the Task Result. + * @return the Task Result. + */ + public TaskResult callShardSyncTask() { + final ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, + leaseRefresher, + initialPositionInStream, + cleanupLeasesUponShardCompletion, + garbageCollectLeases, + ignoreUnexpectedChildShards, + shardSyncIdleTimeMillis, + hierarchicalShardSyncer, + metricsFactory); + final ConsumerTask metricCollectingTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory); + return metricCollectingTask.call(); } - private synchronized boolean checkAndSubmitNextTask() { + /** + * Submit a ShardSyncTask and return if the submission is successful. + * @return if the casting is successful. + */ + public boolean submitShardSyncTask() { + try { + lock.lock(); + return checkAndSubmitNextTask(); + } finally { + lock.unlock(); + } + } + + private boolean checkAndSubmitNextTask() { boolean submittedNewTask = false; if ((future == null) || future.isCancelled() || future.isDone()) { if ((future != null) && future.isDone()) { @@ -140,23 +178,54 @@ public class ShardSyncTaskManager { leaseRefresher, initialPositionInStream, cleanupLeasesUponShardCompletion, + garbageCollectLeases, ignoreUnexpectedChildShards, shardSyncIdleTimeMillis, hierarchicalShardSyncer, metricsFactory), metricsFactory); - future = executorService.submit(currentTask); + future = CompletableFuture.supplyAsync(() -> currentTask.call(), executorService) + .whenComplete((taskResult, exception) -> handlePendingShardSyncs(exception, taskResult)); + + log.info(new ExecutorStateEvent(executorService).message()); + submittedNewTask = true; if (log.isDebugEnabled()) { log.debug("Submitted new {} task.", currentTask.taskType()); } } else { if (log.isDebugEnabled()) { - log.debug("Previous {} task still pending. Not submitting new task.", currentTask.taskType()); + log.debug("Previous {} task still pending. Not submitting new task. " + + "Enqueued a request that will be executed when the current request completes.", currentTask.taskType()); } + shardSyncRequestPending.compareAndSet(false /*expected*/, true /*update*/); } - return submittedNewTask; } + private void handlePendingShardSyncs(Throwable exception, TaskResult taskResult) { + if (exception != null || taskResult.getException() != null) { + log.error("Caught exception running {} task: ", currentTask.taskType(), exception != null ? exception : taskResult.getException()); + } + // Acquire lock here. If shardSyncRequestPending is false in this completionStage and + // submitShardSyncTask is invoked, before completion stage exits (future completes) + // but right after the value of shardSyncRequestPending is checked, it will result in + // shardSyncRequestPending being set to true, but no pending futures to trigger the next + // ShardSyncTask. By executing this stage in a Reentrant lock, we ensure that if the + // previous task is in this completion stage, checkAndSubmitNextTask is not invoked + // until this completionStage exits. + try { + lock.lock(); + if (shardSyncRequestPending.get()) { + shardSyncRequestPending.set(false); + // reset future to null, so next call creates a new one + // without trying to get results from the old future. + future = null; + checkAndSubmitNextTask(); + } + } finally { + lock.unlock(); + } + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java new file mode 100644 index 00000000..9461a18e --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java @@ -0,0 +1,26 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.leases; + +/** + * These are the special fields that will be updated only once during the lifetime of the lease. + * Since these are meta information that will not affect lease ownership or data durability, we allow + * any elected leader or worker to set these fields directly without any conditional checks. + * Note that though HASH_KEY_RANGE will be available during lease initialization in newer versions, we keep this + * for backfilling while rolling forward to newer versions. + */ +public enum UpdateField { + CHILD_SHARDS, HASH_KEY_RANGE +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java index fe31d996..78673f66 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java @@ -38,6 +38,7 @@ import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseRenewer; import software.amazon.kinesis.leases.LeaseTaker; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; @@ -348,8 +349,8 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { @Override public boolean updateLease(final Lease lease, final UUID concurrencyToken, final String operation, - final String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return leaseRenewer.updateLease(lease, concurrencyToken, operation, shardId); + final String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + return leaseRenewer.updateLease(lease, concurrencyToken, operation, singleStreamShardId); } /** @@ -377,9 +378,19 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { return leases.stream().map(DynamoDBLeaseCoordinator::convertLeaseToAssignment).collect(Collectors.toList()); } + /** + * Utility method to convert the basic lease or multistream lease to ShardInfo + * @param lease + * @return ShardInfo + */ public static ShardInfo convertLeaseToAssignment(final Lease lease) { - return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), - lease.checkpoint()); + if (lease instanceof MultiStreamLease) { + return new ShardInfo(((MultiStreamLease) lease).shardId(), lease.concurrencyToken().toString(), lease.parentShardIds(), + lease.checkpoint(), ((MultiStreamLease) lease).streamIdentifier()); + } else { + return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), + lease.checkpoint()); + } } /** diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java index c2ade429..5102bc5e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -17,7 +17,8 @@ package software.amazon.kinesis.leases.dynamodb; import java.time.Duration; import java.util.concurrent.ExecutorService; - +import java.util.concurrent.Executors; +import java.util.function.Function; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; @@ -25,11 +26,16 @@ import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.LeaseCleanupConfig; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.KinesisShardDetector; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseManagementFactory; +import software.amazon.kinesis.leases.LeaseSerializer; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.metrics.MetricsFactory; @@ -44,8 +50,6 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @NonNull private final KinesisAsyncClient kinesisClient; @NonNull - private final String streamName; - @NonNull private final DynamoDbAsyncClient dynamoDBClient; @NonNull private final String tableName; @@ -54,9 +58,13 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @NonNull private final ExecutorService executorService; @NonNull - private final InitialPositionInStreamExtended initialPositionInStream; + private final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer; @NonNull - private final HierarchicalShardSyncer hierarchicalShardSyncer; + private final LeaseSerializer leaseSerializer; + @NonNull + private StreamConfig streamConfig; + + private Function customShardDetectorProvider; private final long failoverTimeMillis; private final long epsilonMillis; @@ -77,6 +85,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { private final TableCreatorCallback tableCreatorCallback; private final Duration dynamoDbRequestTimeout; private final BillingMode billingMode; + private final boolean isMultiStreamMode; + private final LeaseCleanupConfig leaseCleanupConfig; /** * Constructor. @@ -228,7 +238,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { /** * Constructor. - * + * * @param kinesisClient * @param streamName * @param dynamoDBClient @@ -309,6 +319,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param dynamoDbRequestTimeout * @param billingMode */ + @Deprecated public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, @@ -321,13 +332,118 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode) { + + this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, + cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); + } + + /** + * Constructor. + * + * @param kinesisClient + * @param streamConfig + * @param dynamoDBClient + * @param tableName + * @param workerIdentifier + * @param executorService + * @param failoverTimeMillis + * @param epsilonMillis + * @param maxLeasesForWorker + * @param maxLeasesToStealAtOneTime + * @param maxLeaseRenewalThreads + * @param cleanupLeasesUponShardCompletion + * @param ignoreUnexpectedChildShards + * @param shardSyncIntervalMillis + * @param consistentReads + * @param listShardsBackoffTimeMillis + * @param maxListShardsRetryAttempts + * @param maxCacheMissesBeforeReload + * @param listShardsCacheAllowedAgeInSeconds + * @param cacheMissWarningModulus + * @param initialLeaseTableReadCapacity + * @param initialLeaseTableWriteCapacity + * @param deprecatedHierarchicalShardSyncer + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + */ + private DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final StreamConfig streamConfig, + final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, + final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, + final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, + final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, + final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, + final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, + final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, + final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { + this(kinesisClient, dynamoDBClient, tableName, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, + cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, + deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer, + null, false, LeaseManagementConfig.DEFAULT_LEASE_CLEANUP_CONFIG); + this.streamConfig = streamConfig; + } + + /** + * Constructor. + * @param kinesisClient + * @param dynamoDBClient + * @param tableName + * @param workerIdentifier + * @param executorService + * @param failoverTimeMillis + * @param epsilonMillis + * @param maxLeasesForWorker + * @param maxLeasesToStealAtOneTime + * @param maxLeaseRenewalThreads + * @param cleanupLeasesUponShardCompletion + * @param ignoreUnexpectedChildShards + * @param shardSyncIntervalMillis + * @param consistentReads + * @param listShardsBackoffTimeMillis + * @param maxListShardsRetryAttempts + * @param maxCacheMissesBeforeReload + * @param listShardsCacheAllowedAgeInSeconds + * @param cacheMissWarningModulus + * @param initialLeaseTableReadCapacity + * @param initialLeaseTableWriteCapacity + * @param deprecatedHierarchicalShardSyncer + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + * @param leaseSerializer + * @param customShardDetectorProvider + * @param isMultiStreamMode + * @param leaseCleanupConfig + */ + public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, + final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, + final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, + final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, + final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, + final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, + final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, + final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, + final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer, + Function customShardDetectorProvider, boolean isMultiStreamMode, + LeaseCleanupConfig leaseCleanupConfig) { this.kinesisClient = kinesisClient; - this.streamName = streamName; this.dynamoDBClient = dynamoDBClient; this.tableName = tableName; this.workerIdentifier = workerIdentifier; this.executorService = executorService; - this.initialPositionInStream = initialPositionInStream; this.failoverTimeMillis = failoverTimeMillis; this.epsilonMillis = epsilonMillis; this.maxLeasesForWorker = maxLeasesForWorker; @@ -344,10 +460,14 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.cacheMissWarningModulus = cacheMissWarningModulus; this.initialLeaseTableReadCapacity = initialLeaseTableReadCapacity; this.initialLeaseTableWriteCapacity = initialLeaseTableWriteCapacity; - this.hierarchicalShardSyncer = hierarchicalShardSyncer; + this.deprecatedHierarchicalShardSyncer = deprecatedHierarchicalShardSyncer; this.tableCreatorCallback = tableCreatorCallback; this.dynamoDbRequestTimeout = dynamoDbRequestTimeout; this.billingMode = billingMode; + this.leaseSerializer = leaseSerializer; + this.customShardDetectorProvider = customShardDetectorProvider; + this.isMultiStreamMode = isMultiStreamMode; + this.leaseCleanupConfig = leaseCleanupConfig; } @Override @@ -364,29 +484,77 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { metricsFactory); } - @Override + @Override @Deprecated public ShardSyncTaskManager createShardSyncTaskManager(@NonNull final MetricsFactory metricsFactory) { return new ShardSyncTaskManager(this.createShardDetector(), this.createLeaseRefresher(), - initialPositionInStream, + streamConfig.initialPositionInStreamExtended(), + cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, + shardSyncIntervalMillis, + executorService, deprecatedHierarchicalShardSyncer, + metricsFactory); + } + + /** + * Create ShardSyncTaskManager from the streamConfig passed + * @param metricsFactory + * @param streamConfig + * @return ShardSyncTaskManager + */ + @Override + public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { + return new ShardSyncTaskManager(this.createShardDetector(streamConfig), + this.createLeaseRefresher(), + streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, executorService, - hierarchicalShardSyncer, + new HierarchicalShardSyncer(isMultiStreamMode, streamConfig.streamIdentifier().toString()), metricsFactory); } @Override public DynamoDBLeaseRefresher createLeaseRefresher() { - return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, new DynamoDBLeaseSerializer(), consistentReads, + return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, leaseSerializer, consistentReads, tableCreatorCallback, dynamoDbRequestTimeout, billingMode); } @Override + @Deprecated public ShardDetector createShardDetector() { - return new KinesisShardDetector(kinesisClient, streamName, listShardsBackoffTimeMillis, - maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, - cacheMissWarningModulus, dynamoDbRequestTimeout); + return new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), + listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, + maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout); + } + + /** + * KinesisShardDetector supports reading from service only using streamName. Support for accountId and + * stream creation epoch is yet to be provided. + * @param streamConfig + * @return ShardDetector + */ + @Override + public ShardDetector createShardDetector(StreamConfig streamConfig) { + return customShardDetectorProvider != null ? customShardDetectorProvider.apply(streamConfig) : + new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, + cacheMissWarningModulus, dynamoDbRequestTimeout); + } + + /** + * LeaseCleanupManager cleans up leases in the lease table for shards which have either expired past the + * stream's retention period or have been completely processed. + * @param metricsFactory + * @return LeaseCleanupManager + */ + @Override + public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) { + return new LeaseCleanupManager(createLeaseCoordinator(metricsFactory), + metricsFactory, Executors.newSingleThreadScheduledExecutor(), + cleanupLeasesUponShardCompletion, leaseCleanupConfig.leaseCleanupIntervalMillis(), + leaseCleanupConfig.completedLeaseCleanupIntervalMillis(), + leaseCleanupConfig.garbageLeaseCleanupIntervalMillis()); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java index 10526ea6..8002eacc 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java @@ -14,6 +14,8 @@ */ package software.amazon.kinesis.leases.dynamodb; +import com.google.common.collect.ImmutableMap; + import java.time.Duration; import java.util.ArrayList; import java.util.List; @@ -21,24 +23,44 @@ import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; - import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; -import software.amazon.awssdk.services.dynamodb.model.*; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException; +import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DeleteItemRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.model.DynamoDbException; +import software.amazon.awssdk.services.dynamodb.model.GetItemRequest; +import software.amazon.awssdk.services.dynamodb.model.GetItemResponse; +import software.amazon.awssdk.services.dynamodb.model.LimitExceededException; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughput; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputExceededException; +import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; +import software.amazon.awssdk.services.dynamodb.model.ResourceInUseException; +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; +import software.amazon.awssdk.services.dynamodb.model.ScanRequest; +import software.amazon.awssdk.services.dynamodb.model.ScanResponse; +import software.amazon.awssdk.services.dynamodb.model.TableStatus; +import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseSerializer; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.retrieval.AWSExceptionManager; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import software.amazon.awssdk.services.dynamodb.model.BillingMode; /** * An implementation of {@link LeaseRefresher} that uses DynamoDB. @@ -58,6 +80,9 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { private boolean newTableCreated = false; + private static final String STREAM_NAME = "streamName"; + private static final String DDB_STREAM_NAME = ":streamName"; + /** * Constructor. * @@ -264,12 +289,21 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { return System.currentTimeMillis() - startTime; } + /** + * {@inheritDoc} + */ + @Override + public List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, + InvalidStateException, ProvisionedThroughputException { + return list( null, streamIdentifier); + } + /** * {@inheritDoc} */ @Override public List listLeases() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(null); + return list(null, null); } /** @@ -278,22 +312,50 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(1).isEmpty(); + return list(1, 1, null).isEmpty(); } /** * List with the given page size. Package access for integration testing. * * @param limit number of items to consider at a time - used by integration tests to force paging. + * @param streamIdentifier streamIdentifier for multi-stream mode. Can be null. * @return list of leases * @throws InvalidStateException if table does not exist * @throws DependencyException if DynamoDB scan fail in an unexpected way * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity */ - List list(Integer limit) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + List list(Integer limit, StreamIdentifier streamIdentifier) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + return list(limit, Integer.MAX_VALUE, streamIdentifier); + } + + /** + * List with the given page size. Package access for integration testing. + * + * @param limit number of items to consider at a time - used by integration tests to force paging. + * @param maxPages mad paginated scan calls + * @param streamIdentifier streamIdentifier for multi-stream mode. Can be null. + * @return list of leases + * @throws InvalidStateException if table does not exist + * @throws DependencyException if DynamoDB scan fail in an unexpected way + * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity + */ + private List list(Integer limit, Integer maxPages, StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, + ProvisionedThroughputException { + log.debug("Listing leases from table {}", table); ScanRequest.Builder scanRequestBuilder = ScanRequest.builder().tableName(table); + + if (streamIdentifier != null) { + final Map expressionAttributeValues = ImmutableMap.of( + DDB_STREAM_NAME, AttributeValue.builder().s(streamIdentifier.serialize()).build() + ); + scanRequestBuilder = scanRequestBuilder.filterExpression(STREAM_NAME + " = " + DDB_STREAM_NAME) + .expressionAttributeValues(expressionAttributeValues); + } + if (limit != null) { scanRequestBuilder = scanRequestBuilder.limit(limit); } @@ -315,7 +377,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } Map lastEvaluatedKey = scanResult.lastEvaluatedKey(); - if (CollectionUtils.isNullOrEmpty(lastEvaluatedKey)) { + if (CollectionUtils.isNullOrEmpty(lastEvaluatedKey) || --maxPages <= 0) { // Signify that we're done. scanResult = null; log.debug("lastEvaluatedKey was null - scan finished."); @@ -634,14 +696,40 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { return true; } + @Override + public void updateLeaseWithMetaInfo(Lease lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + log.debug("Updating lease without expectation {}", lease); + final AWSExceptionManager exceptionManager = createExceptionManager(); + exceptionManager.add(ConditionalCheckFailedException.class, t -> t); + Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); + UpdateItemRequest request = UpdateItemRequest.builder().tableName(table).key(serializer.getDynamoHashKey(lease)) + .expected(serializer.getDynamoExistentExpectation(lease.leaseKey())) + .attributeUpdates(updates).build(); + try { + try { + FutureUtils.resolveOrCancelFuture(dynamoDBClient.updateItem(request), dynamoDbRequestTimeout); + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } catch (InterruptedException e) { + throw new DependencyException(e); + } + } catch (ConditionalCheckFailedException e) { + log.warn("Lease update failed for lease with key {} because the lease did not exist at the time of the update", + lease.leaseKey(), e); + } catch (DynamoDbException | TimeoutException e) { + throw convertAndRethrowExceptions("update", lease.leaseKey(), e); + } + } + /** * {@inheritDoc} */ @Override - public ExtendedSequenceNumber getCheckpoint(String shardId) + public ExtendedSequenceNumber getCheckpoint(String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException { ExtendedSequenceNumber checkpoint = null; - Lease lease = getLease(shardId); + Lease lease = getLease(leaseKey); if (lease != null) { checkpoint = lease.checkpoint(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java index 04d987a2..e457b5ec 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java @@ -36,9 +36,11 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseRenewer; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -269,7 +271,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { * {@inheritDoc} */ @Override - public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String shardId) + public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { verifyNotNull(lease, "lease cannot be null"); verifyNotNull(lease.leaseKey(), "leaseKey cannot be null"); @@ -296,8 +298,12 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { } final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation); - if (StringUtils.isNotEmpty(shardId)) { - MetricsUtil.addShardId(scope, shardId); + if (lease instanceof MultiStreamLease) { + MetricsUtil.addStreamId(scope, + StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())); + MetricsUtil.addShardId(scope, ((MultiStreamLease) lease).shardId()); + } else if (StringUtils.isNotEmpty(singleStreamShardId)) { + MetricsUtil.addShardId(scope, singleStreamShardId); } long startTime = System.currentTimeMillis(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java index b97738ca..64a7840c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java @@ -30,10 +30,13 @@ import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue; import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; import software.amazon.awssdk.services.dynamodb.model.KeyType; import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.leases.DynamoUtils; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseSerializer; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; /** @@ -50,7 +53,11 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String CHECKPOINT_SUBSEQUENCE_NUMBER_KEY = "checkpointSubSequenceNumber"; private static final String PENDING_CHECKPOINT_SEQUENCE_KEY = "pendingCheckpoint"; private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber"; + private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; + private static final String CHILD_SHARD_IDS_KEY = "childShardIds"; + private static final String STARTING_HASH_KEY = "startingHashKey"; + private static final String ENDING_HASH_KEY = "endingHashKey"; @Override public Map toDynamoRecord(final Lease lease) { @@ -69,39 +76,65 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { if (lease.parentShardIds() != null && !lease.parentShardIds().isEmpty()) { result.put(PARENT_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.parentShardIds())); } + if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { + result.put(CHILD_SHARD_IDS_KEY, DynamoUtils.createAttributeValue(lease.childShardIds())); + } if (lease.pendingCheckpoint() != null && !lease.pendingCheckpoint().sequenceNumber().isEmpty()) { result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().sequenceNumber())); result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().subSequenceNumber())); } + if (lease.pendingCheckpointState() != null) { + result.put(PENDING_CHECKPOINT_STATE_KEY, DynamoUtils.createAttributeValue(lease.checkpoint().subSequenceNumber())); + } + + if(lease.hashKeyRangeForLease() != null) { + result.put(STARTING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey())); + result.put(ENDING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey())); + } + return result; } @Override public Lease fromDynamoRecord(final Map dynamoRecord) { - Lease result = new Lease(); - result.leaseKey(DynamoUtils.safeGetString(dynamoRecord, LEASE_KEY_KEY)); - result.leaseOwner(DynamoUtils.safeGetString(dynamoRecord, LEASE_OWNER_KEY)); - result.leaseCounter(DynamoUtils.safeGetLong(dynamoRecord, LEASE_COUNTER_KEY)); + final Lease result = new Lease(); + return fromDynamoRecord(dynamoRecord, result); + } - result.ownerSwitchesSinceCheckpoint(DynamoUtils.safeGetLong(dynamoRecord, OWNER_SWITCHES_KEY)); - result.checkpoint( + @Override + public Lease fromDynamoRecord(Map dynamoRecord, Lease leaseToUpdate) { + leaseToUpdate.leaseKey(DynamoUtils.safeGetString(dynamoRecord, LEASE_KEY_KEY)); + leaseToUpdate.leaseOwner(DynamoUtils.safeGetString(dynamoRecord, LEASE_OWNER_KEY)); + leaseToUpdate.leaseCounter(DynamoUtils.safeGetLong(dynamoRecord, LEASE_COUNTER_KEY)); + + leaseToUpdate.ownerSwitchesSinceCheckpoint(DynamoUtils.safeGetLong(dynamoRecord, OWNER_SWITCHES_KEY)); + leaseToUpdate.checkpoint( new ExtendedSequenceNumber( DynamoUtils.safeGetString(dynamoRecord, CHECKPOINT_SEQUENCE_NUMBER_KEY), DynamoUtils.safeGetLong(dynamoRecord, CHECKPOINT_SUBSEQUENCE_NUMBER_KEY)) ); - result.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY)); + leaseToUpdate.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY)); + leaseToUpdate.childShardIds(DynamoUtils.safeGetSS(dynamoRecord, CHILD_SHARD_IDS_KEY)); if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) { - result.pendingCheckpoint( + leaseToUpdate.pendingCheckpoint( new ExtendedSequenceNumber( DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY), DynamoUtils.safeGetLong(dynamoRecord, PENDING_CHECKPOINT_SUBSEQUENCE_KEY)) ); } - return result; + leaseToUpdate.pendingCheckpointState(DynamoUtils.safeGetByteArray(dynamoRecord, PENDING_CHECKPOINT_STATE_KEY)); + + final String startingHashKey, endingHashKey; + if (!Strings.isNullOrEmpty(startingHashKey = DynamoUtils.safeGetString(dynamoRecord, STARTING_HASH_KEY)) + && !Strings.isNullOrEmpty(endingHashKey = DynamoUtils.safeGetString(dynamoRecord, ENDING_HASH_KEY))) { + leaseToUpdate.hashKeyRange(HashKeyRangeForLease.deserialize(startingHashKey, endingHashKey)); + } + + return leaseToUpdate; } @Override @@ -159,6 +192,19 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { return result; } + @Override + public Map getDynamoExistentExpectation(String leaseKey) { + Map result = new HashMap<>(); + + ExpectedAttributeValue expectedAV = ExpectedAttributeValue.builder() + .exists(true) + .value(DynamoUtils.createAttributeValue(leaseKey)) + .build(); + result.put(LEASE_KEY_KEY, expectedAV); + + return result; + } + @Override public Map getDynamoLeaseCounterUpdate(final Lease lease) { return getDynamoLeaseCounterUpdate(lease.leaseCounter()); @@ -198,7 +244,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { return result; } - private AttributeValueUpdate putUpdate(AttributeValue attributeValue) { + protected AttributeValueUpdate putUpdate(AttributeValue attributeValue) { return AttributeValueUpdate.builder().value(attributeValue).action(AttributeAction.PUT).build(); } @@ -216,6 +262,45 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); } + + if (lease.pendingCheckpointState() != null) { + result.put(PENDING_CHECKPOINT_STATE_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.pendingCheckpointState()))); + } else { + result.put(PENDING_CHECKPOINT_STATE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); + } + + + if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { + result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); + } + + if(lease.hashKeyRangeForLease() != null) { + result.put(STARTING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey()))); + result.put(ENDING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey()))); + } + + return result; + } + + @Override + public Map getDynamoUpdateLeaseUpdate(Lease lease, + UpdateField updateField) { + Map result = new HashMap<>(); + switch (updateField) { + case CHILD_SHARDS: + if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { + result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); + } + break; + case HASH_KEY_RANGE: + if (lease.hashKeyRangeForLease() != null) { + result.put(STARTING_HASH_KEY, putUpdate( + DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey()))); + result.put(ENDING_HASH_KEY, putUpdate( + DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey()))); + } + break; + } return result; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java index d79646e7..4249f32b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java @@ -25,7 +25,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; - +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -36,8 +36,8 @@ import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.metrics.MetricsFactory; -import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; /** @@ -48,6 +48,7 @@ import software.amazon.kinesis.metrics.MetricsUtil; public class DynamoDBLeaseTaker implements LeaseTaker { private static final int TAKE_RETRIES = 3; private static final int SCAN_RETRIES = 1; + private long veryOldLeaseDurationNanosMultiplier = 3; // See note on TAKE_LEASES_DIMENSION(Callable) for why we have this callable. private static final Callable SYSTEM_CLOCK_CALLABLE = System::nanoTime; @@ -95,6 +96,18 @@ public class DynamoDBLeaseTaker implements LeaseTaker { return this; } + /** + * Overrides the default very old lease duration nanos multiplier to increase the threshold for taking very old leases. + * Setting this to a higher value than 3 will increase the threshold for very old lease taking. + * + * @param veryOldLeaseDurationNanosMultipler Very old lease duration multiplier for adjusting very old lease taking. + * @return LeaseTaker + */ + public DynamoDBLeaseTaker withVeryOldLeaseDurationNanosMultipler(long veryOldLeaseDurationNanosMultipler) { + this.veryOldLeaseDurationNanosMultiplier = veryOldLeaseDurationNanosMultipler; + return this; + } + /** * Max leases to steal from a more loaded Worker at one time (for load balancing). * Setting this to a higher number can allow for faster load convergence (e.g. during deployments, cold starts), @@ -329,31 +342,39 @@ public class DynamoDBLeaseTaker implements LeaseTaker { Set leasesToTake = new HashSet<>(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION); MetricsUtil.addWorkerIdentifier(scope, workerIdentifier); + List veryOldLeases = new ArrayList<>(); + + int numLeases = 0; + int numWorkers = 0; + int numLeasesToReachTarget = 0; + int leaseSpillover = 0; + int veryOldLeaseCount = 0; try { - int numLeases = allLeases.size(); - int numWorkers = leaseCounts.size(); + numLeases = allLeases.size(); + numWorkers = leaseCounts.size(); if (numLeases == 0) { // If there are no leases, I shouldn't try to take any. return leasesToTake; } + int target; if (numWorkers >= numLeases) { // If we have n leases and n or more workers, each worker can have up to 1 lease, including myself. target = 1; } else { - /* - * numWorkers must be < numLeases. - * - * Our target for each worker is numLeases / numWorkers (+1 if numWorkers doesn't evenly divide numLeases) - */ + /* + * numWorkers must be < numLeases. + * + * Our target for each worker is numLeases / numWorkers (+1 if numWorkers doesn't evenly divide numLeases) + */ target = numLeases / numWorkers + (numLeases % numWorkers == 0 ? 0 : 1); // Spill over is the number of leases this worker should have claimed, but did not because it would // exceed the max allowed for this worker. - int leaseSpillover = Math.max(0, target - maxLeasesForWorker); + leaseSpillover = Math.max(0, target - maxLeasesForWorker); if (target > maxLeasesForWorker) { log.warn( "Worker {} target is {} leases and maxLeasesForWorker is {}. Resetting target to {}," @@ -362,11 +383,29 @@ public class DynamoDBLeaseTaker implements LeaseTaker { workerIdentifier, target, maxLeasesForWorker, maxLeasesForWorker, leaseSpillover); target = maxLeasesForWorker; } - scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY); } int myCount = leaseCounts.get(workerIdentifier); - int numLeasesToReachTarget = target - myCount; + numLeasesToReachTarget = target - myCount; + + int currentLeaseCount = leaseCounts.get(workerIdentifier); + // If there are leases that have been expired for an extended period of + // time, take them with priority, disregarding the target (computed + // later) but obeying the maximum limit per worker. + veryOldLeases = allLeases.values().stream() + .filter(lease -> System.nanoTime() - lease.lastCounterIncrementNanos() + > veryOldLeaseDurationNanosMultiplier * leaseDurationNanos) + .collect(Collectors.toList()); + + if (!veryOldLeases.isEmpty()) { + Collections.shuffle(veryOldLeases); + veryOldLeaseCount = Math.max(0, Math.min(maxLeasesForWorker - currentLeaseCount, veryOldLeases.size())); + HashSet result = new HashSet<>(veryOldLeases.subList(0, veryOldLeaseCount)); + if (veryOldLeaseCount > 0) { + log.info("Taking leases that have been expired for a long time: {}", result); + } + return result; + } if (numLeasesToReachTarget <= 0) { // If we don't need anything, return the empty set. @@ -376,7 +415,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { // Shuffle expiredLeases so workers don't all try to contend for the same leases. Collections.shuffle(expiredLeases); - int originalExpiredLeasesSize = expiredLeases.size(); if (expiredLeases.size() > 0) { // If we have expired leases, get up to leases from expiredLeases for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) { @@ -397,16 +435,19 @@ public class DynamoDBLeaseTaker implements LeaseTaker { log.info( "Worker {} saw {} total leases, {} available leases, {} " + "workers. Target is {} leases, I have {} leases, I will take {} leases", - workerIdentifier, numLeases, originalExpiredLeasesSize, numWorkers, target, myCount, + workerIdentifier, numLeases, expiredLeases.size(), numWorkers, target, myCount, leasesToTake.size()); } - scope.addData("TotalLeases", numLeases, StandardUnit.COUNT, MetricsLevel.DETAILED); - scope.addData("ExpiredLeases", originalExpiredLeasesSize, StandardUnit.COUNT, MetricsLevel.SUMMARY); - scope.addData("NumWorkers", numWorkers, StandardUnit.COUNT, MetricsLevel.SUMMARY); - scope.addData("NeededLeases", numLeasesToReachTarget, StandardUnit.COUNT, MetricsLevel.DETAILED); - scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED); } finally { + scope.addData("ExpiredLeases", expiredLeases.size(), StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED); + scope.addData("NeededLeases", Math.max(numLeasesToReachTarget, 0), StandardUnit.COUNT, MetricsLevel.DETAILED); + scope.addData("NumWorkers", numWorkers, StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("TotalLeases", numLeases, StandardUnit.COUNT, MetricsLevel.DETAILED); + scope.addData("VeryOldLeases", veryOldLeaseCount, StandardUnit.COUNT, MetricsLevel.SUMMARY); + MetricsUtil.endScope(scope); } @@ -499,10 +540,13 @@ public class DynamoDBLeaseTaker implements LeaseTaker { */ private Map computeLeaseCounts(List expiredLeases) { Map leaseCounts = new HashMap<>(); + // The set will give much faster lookup than the original list, an + // important optimization when the list is large + Set expiredLeasesSet = new HashSet<>(expiredLeases); // Compute the number of leases per worker by looking through allLeases and ignoring leases that have expired. for (Lease lease : allLeases.values()) { - if (!expiredLeases.contains(lease)) { + if (!expiredLeasesSet.contains(lease)) { String leaseOwner = lease.leaseOwner(); Integer oldCount = leaseCounts.get(leaseOwner); if (oldCount == null) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java new file mode 100644 index 00000000..78c9c6c4 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java @@ -0,0 +1,63 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases.dynamodb; + +import lombok.NoArgsConstructor; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; +import software.amazon.kinesis.leases.DynamoUtils; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.MultiStreamLease; + +import java.util.Map; + +import static software.amazon.kinesis.leases.MultiStreamLease.validateAndCast; + +@NoArgsConstructor +public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer { + + // Keeping the stream id as "streamName" for legacy reasons. + private static final String STREAM_ID_KEY = "streamName"; + private static final String SHARD_ID_KEY = "shardId"; + + @Override + public Map toDynamoRecord(Lease lease) { + final MultiStreamLease multiStreamLease = validateAndCast(lease); + final Map result = super.toDynamoRecord(multiStreamLease); + result.put(STREAM_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier())); + result.put(SHARD_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.shardId())); + return result; + } + + @Override + public MultiStreamLease fromDynamoRecord(Map dynamoRecord) { + final MultiStreamLease multiStreamLease = (MultiStreamLease) super + .fromDynamoRecord(dynamoRecord, new MultiStreamLease()); + multiStreamLease.streamIdentifier(DynamoUtils.safeGetString(dynamoRecord, STREAM_ID_KEY)); + multiStreamLease.shardId(DynamoUtils.safeGetString(dynamoRecord, SHARD_ID_KEY)); + return multiStreamLease; + } + + + @Override + public Map getDynamoUpdateLeaseUpdate(Lease lease) { + final MultiStreamLease multiStreamLease = validateAndCast(lease); + final Map result = super.getDynamoUpdateLeaseUpdate(multiStreamLease); + result.put(STREAM_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier()))); + result.put(SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.shardId()))); + return result; + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java new file mode 100644 index 00000000..ba97ab08 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java @@ -0,0 +1,27 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.leases.exceptions; + +/** + * Exception type for all exceptions thrown by the customer implemented code. + */ +public class CustomerApplicationException extends Exception { + + public CustomerApplicationException(Throwable e) { super(e);} + + public CustomerApplicationException(String message, Throwable e) { super(message, e);} + + public CustomerApplicationException(String message) { super(message);} +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java new file mode 100644 index 00000000..2d3d0c2f --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java @@ -0,0 +1,53 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases.exceptions; + +import lombok.EqualsAndHashCode; +import lombok.Value; +import lombok.experimental.Accessors; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.ShardDetector; +import software.amazon.kinesis.leases.ShardInfo; + +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * Helper class for cleaning up leases. + */ +@Accessors(fluent = true) +@Value +@EqualsAndHashCode(exclude = {"queueEntryTime"}) +public class LeasePendingDeletion { + private final StreamIdentifier streamIdentifier; + private final Lease lease; + private final ShardInfo shardInfo; + private final ShardDetector shardDetector; + + /** + * Discovers the child shards for this lease. + * @return + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + public Set getChildShardsFromService() throws InterruptedException, ExecutionException, TimeoutException { + return shardDetector.getChildShards(shardInfo.shardId()).stream().map(c -> c.shardId()).collect(Collectors.toSet()); + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java index 4e9245f6..f7ec12c5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java @@ -19,6 +19,7 @@ import software.amazon.kinesis.metrics.MetricsScope; @Deprecated public class ShardSyncer { private static final HierarchicalShardSyncer HIERARCHICAL_SHARD_SYNCER = new HierarchicalShardSyncer(); + private static final boolean garbageCollectLeases = true; /** *

NOTE: This method is deprecated and will be removed in a future release.

@@ -26,7 +27,6 @@ public class ShardSyncer { * @param shardDetector * @param leaseRefresher * @param initialPosition - * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards * @param scope * @throws DependencyException @@ -37,10 +37,9 @@ public class ShardSyncer { @Deprecated public static synchronized void checkAndCreateLeasesForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, - final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, - KinesisClientLibIOException { - HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope); + final boolean ignoreUnexpectedChildShards, final MetricsScope scope) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { + HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, + scope, ignoreUnexpectedChildShards, leaseRefresher.isLeaseTableEmpty()); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java index 4ea4212e..5f1ee18c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java @@ -54,28 +54,29 @@ public class BlockOnParentShardTask implements ConsumerTask { @Override public TaskResult call() { Exception exception = null; - + final String shardInfoId = ShardInfo.getLeaseKey(shardInfo); try { boolean blockedOnParentShard = false; for (String shardId : shardInfo.parentShardIds()) { - Lease lease = leaseRefresher.getLease(shardId); + final String leaseKey = ShardInfo.getLeaseKey(shardInfo, shardId); + final Lease lease = leaseRefresher.getLease(leaseKey); if (lease != null) { ExtendedSequenceNumber checkpoint = lease.checkpoint(); if ((checkpoint == null) || (!checkpoint.equals(ExtendedSequenceNumber.SHARD_END))) { - log.debug("Shard {} is not yet done. Its current checkpoint is {}", shardId, checkpoint); + log.debug("Shard {} is not yet done. Its current checkpoint is {}", shardInfoId, checkpoint); blockedOnParentShard = true; exception = new BlockedOnParentShardException("Parent shard not yet done"); break; } else { - log.debug("Shard {} has been completely processed.", shardId); + log.debug("Shard {} has been completely processed.", shardInfoId); } } else { - log.info("No lease found for shard {}. Not blocking on completion of this shard.", shardId); + log.info("No lease found for shard {}. Not blocking on completion of this shard.", shardInfoId); } } if (!blockedOnParentShard) { - log.info("No need to block on parents {} of shard {}", shardInfo.parentShardIds(), shardInfo.shardId()); + log.info("No need to block on parents {} of shard {}", shardInfo.parentShardIds(), shardInfoId); return new TaskResult(null); } } catch (Exception e) { @@ -85,7 +86,7 @@ public class BlockOnParentShardTask implements ConsumerTask { try { Thread.sleep(parentShardPollIntervalMillis); } catch (InterruptedException e) { - log.error("Sleep interrupted when waiting on parent shard(s) of {}", shardInfo.shardId(), e); + log.error("Sleep interrupted when waiting on parent shard(s) of {}", shardInfoId, e); } return new TaskResult(exception); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java index bb1788b2..4d894d94 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java @@ -146,7 +146,7 @@ class ConsumerStates { @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { - return ShardConsumerState.SHUTDOWN_COMPLETE.consumerState(); + return ShardConsumerState.SHUTTING_DOWN.consumerState(); } @Override @@ -496,7 +496,10 @@ class ConsumerStates { argument.taskBackoffTimeMillis(), argument.recordsPublisher(), argument.hierarchicalShardSyncer(), - argument.metricsFactory()); + argument.metricsFactory(), + input == null ? null : input.childShards(), + argument.streamIdentifier(), + argument.leaseCleanupManager()); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java index fdb0e947..4108dd9b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java @@ -21,6 +21,7 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.metrics.MetricsFactory; @@ -75,9 +76,10 @@ public class InitializeTask implements ConsumerTask { try { log.debug("Initializing ShardId {}", shardInfo); - Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(shardInfo.shardId()); + final String leaseKey = ShardInfo.getLeaseKey(shardInfo); + Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(leaseKey); ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.checkpoint(); - log.debug("[{}]: Checkpoint: {} -- Initial Position: {}", shardInfo.shardId(), initialCheckpoint, + log.debug("[{}]: Checkpoint: {} -- Initial Position: {}", leaseKey, initialCheckpoint, initialPositionInStream); cache.start(initialCheckpoint, initialPositionInStream); @@ -90,6 +92,7 @@ public class InitializeTask implements ConsumerTask { .shardId(shardInfo.shardId()) .extendedSequenceNumber(initialCheckpoint) .pendingCheckpointSequenceNumber(initialCheckpointObject.pendingCheckpoint()) + .pendingCheckpointState(initialCheckpointObject.pendingCheckpointState()) .build(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java index 6c223650..6c52e0de 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java @@ -23,6 +23,7 @@ import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; @@ -60,6 +61,7 @@ public class ProcessTask implements ConsumerTask { private final ProcessRecordsInput processRecordsInput; private final MetricsFactory metricsFactory; private final AggregatorUtil aggregatorUtil; + private final String shardInfoId; public ProcessTask(@NonNull ShardInfo shardInfo, @NonNull ShardRecordProcessor shardRecordProcessor, @@ -74,6 +76,7 @@ public class ProcessTask implements ConsumerTask { @NonNull AggregatorUtil aggregatorUtil, @NonNull MetricsFactory metricsFactory) { this.shardInfo = shardInfo; + this.shardInfoId = ShardInfo.getLeaseKey(shardInfo); this.shardRecordProcessor = shardRecordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; this.backoffTimeMillis = backoffTimeMillis; @@ -106,6 +109,8 @@ public class ProcessTask implements ConsumerTask { @Override public TaskResult call() { final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION); + shardInfo.streamIdentifierSerOpt() + .ifPresent(streamId -> MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(streamId))); MetricsUtil.addShardId(scope, shardInfo.shardId()); long startTimeMillis = System.currentTimeMillis(); boolean success = false; @@ -121,7 +126,7 @@ public class ProcessTask implements ConsumerTask { } if (processRecordsInput.isAtShardEnd() && processRecordsInput.records().isEmpty()) { - log.info("Reached end of shard {} and have no records to process", shardInfo.shardId()); + log.info("Reached end of shard {} and have no records to process", shardInfoId); return new TaskResult(null, true); } @@ -142,13 +147,13 @@ public class ProcessTask implements ConsumerTask { } success = true; } catch (RuntimeException e) { - log.error("ShardId {}: Caught exception: ", shardInfo.shardId(), e); + log.error("ShardId {}: Caught exception: ", shardInfoId, e); exception = e; backoff(); } if (processRecordsInput.isAtShardEnd()) { - log.info("Reached end of shard {}, and processed {} records", shardInfo.shardId(), processRecordsInput.records().size()); + log.info("Reached end of shard {}, and processed {} records", shardInfoId, processRecordsInput.records().size()); return new TaskResult(null, true); } return new TaskResult(exception); @@ -174,7 +179,7 @@ public class ProcessTask implements ConsumerTask { try { Thread.sleep(this.backoffTimeMillis); } catch (InterruptedException ie) { - log.debug("{}: Sleep was interrupted", shardInfo.shardId(), ie); + log.debug("{}: Sleep was interrupted", shardInfoId, ie); } } @@ -188,20 +193,22 @@ public class ProcessTask implements ConsumerTask { */ private void callProcessRecords(ProcessRecordsInput input, List records) { log.debug("Calling application processRecords() with {} records from {}", records.size(), - shardInfo.shardId()); + shardInfoId); final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records).cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime()) .checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION); + shardInfo.streamIdentifierSerOpt() + .ifPresent(streamId -> MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(streamId))); MetricsUtil.addShardId(scope, shardInfo.shardId()); final long startTime = System.currentTimeMillis(); try { shardRecordProcessor.processRecords(processRecordsInput); } catch (Exception e) { log.error("ShardId {}: Application processRecords() threw an exception when processing shard ", - shardInfo.shardId(), e); - log.error("ShardId {}: Skipping over the following data records: {}", shardInfo.shardId(), records); + shardInfoId, e); + log.error("ShardId {}: Skipping over the following data records: {}", shardInfoId, records); } finally { MetricsUtil.addLatency(scope, RECORD_PROCESSOR_PROCESS_RECORDS_METRIC, startTime, MetricsLevel.SUMMARY); MetricsUtil.endScope(scope); @@ -226,17 +233,17 @@ public class ProcessTask implements ConsumerTask { * the time when the task started */ private void handleNoRecords(long startTimeMillis) { - log.debug("Kinesis didn't return any records for shard {}", shardInfo.shardId()); + log.debug("Kinesis didn't return any records for shard {}", shardInfoId); long sleepTimeMillis = idleTimeInMilliseconds - (System.currentTimeMillis() - startTimeMillis); if (sleepTimeMillis > 0) { sleepTimeMillis = Math.max(sleepTimeMillis, idleTimeInMilliseconds); try { log.debug("Sleeping for {} ms since there were no new records in shard {}", sleepTimeMillis, - shardInfo.shardId()); + shardInfoId); Thread.sleep(sleepTimeMillis); } catch (InterruptedException e) { - log.debug("ShardId {}: Sleep was interrupted", shardInfo.shardId()); + log.debug("ShardId {}: Sleep was interrupted", shardInfoId); } } } @@ -273,8 +280,8 @@ public class ProcessTask implements ConsumerTask { if (extendedSequenceNumber.compareTo(lastCheckpointValue) <= 0) { recordIterator.remove(); - log.debug("removing record with ESN {} because the ESN is <= checkpoint ({})", extendedSequenceNumber, - lastCheckpointValue); + log.debug("{} : removing record with ESN {} because the ESN is <= checkpoint ({})", shardInfoId, + extendedSequenceNumber, lastCheckpointValue); continue; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java index 99a680bf..b6e7c068 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.lifecycle; import java.time.Duration; import java.time.Instant; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -32,6 +33,7 @@ import lombok.Getter; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; import software.amazon.kinesis.leases.ShardInfo; @@ -62,6 +64,7 @@ public class ShardConsumer { private final Function taskMetricsDecorator; private final int bufferSize; private final TaskExecutionListener taskExecutionListener; + private final String streamIdentifier; private ConsumerTask currentTask; private TaskOutcome taskOutcome; @@ -85,6 +88,8 @@ public class ShardConsumer { private final ShardConsumerSubscriber subscriber; + private ProcessRecordsInput shardEndProcessRecordsInput; + @Deprecated public ShardConsumer(RecordsPublisher recordsPublisher, ExecutorService executorService, ShardInfo shardInfo, Optional logWarningForTaskAfterMillis, ShardConsumerArgument shardConsumerArgument, @@ -124,6 +129,7 @@ public class ShardConsumer { this.recordsPublisher = recordsPublisher; this.executorService = executorService; this.shardInfo = shardInfo; + this.streamIdentifier = shardInfo.streamIdentifierSerOpt().orElse("single_stream_mode"); this.shardConsumerArgument = shardConsumerArgument; this.logWarningForTaskAfterMillis = logWarningForTaskAfterMillis; this.taskExecutionListener = taskExecutionListener; @@ -146,6 +152,7 @@ public class ShardConsumer { processData(input); if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.SHARD_END); + shardEndProcessRecordsInput = input; subscription.cancel(); return; } @@ -208,8 +215,8 @@ public class ShardConsumer { } Throwable dispatchFailure = subscriber.getAndResetDispatchFailure(); if (dispatchFailure != null) { - log.warn("Exception occurred while dispatching incoming data. The incoming data has been skipped", - dispatchFailure); + log.warn("{} : Exception occurred while dispatching incoming data. The incoming data has been skipped", + streamIdentifier, dispatchFailure); return dispatchFailure; } @@ -238,7 +245,7 @@ public class ShardConsumer { Instant now = Instant.now(); Duration timeSince = Duration.between(subscriber.lastDataArrival(), now); if (timeSince.toMillis() > value) { - log.warn("Last time data arrived: {} ({})", lastDataArrival, timeSince); + log.warn("{} : Last time data arrived: {} ({})", streamIdentifier, lastDataArrival, timeSince); } } }); @@ -250,11 +257,11 @@ public class ShardConsumer { if (taken != null) { String message = longRunningTaskMessage(taken); if (log.isDebugEnabled()) { - log.debug("{} Not submitting new task.", message); + log.debug("{} : {} Not submitting new task.", streamIdentifier, message); } logWarningForTaskAfterMillis.ifPresent(value -> { if (taken.toMillis() > value) { - log.warn(message); + log.warn("{} : {}", streamIdentifier, message); } }); } @@ -303,7 +310,7 @@ public class ShardConsumer { return true; } - executeTask(null); + executeTask(shardEndProcessRecordsInput); return false; } }, executorService); @@ -358,7 +365,7 @@ public class ShardConsumer { nextState = currentState.failureTransition(); break; default: - log.error("No handler for outcome of {}", outcome.name()); + log.error("{} : No handler for outcome of {}", streamIdentifier, outcome.name()); nextState = currentState.failureTransition(); break; } @@ -382,9 +389,9 @@ public class ShardConsumer { Exception taskException = taskResult.getException(); if (taskException instanceof BlockedOnParentShardException) { // No need to log the stack trace for this exception (it is very specific). - log.debug("Shard {} is blocked on completion of parent shard.", shardInfo.shardId()); + log.debug("{} : Shard {} is blocked on completion of parent shard.", streamIdentifier, shardInfo.shardId()); } else { - log.debug("Caught exception running {} task: ", currentTask.taskType(), taskResult.getException()); + log.debug("{} : Caught exception running {} task: ", streamIdentifier, currentTask.taskType(), taskResult.getException()); } } } @@ -411,10 +418,10 @@ public class ShardConsumer { * @return true if shutdown is complete (false if shutdown is still in progress) */ public boolean leaseLost() { - log.debug("Shutdown({}): Lease lost triggered.", shardInfo.shardId()); + log.debug("{} : Shutdown({}): Lease lost triggered.", streamIdentifier, shardInfo.shardId()); if (subscriber != null) { subscriber.cancel(); - log.debug("Shutdown({}): Subscriber cancelled.", shardInfo.shardId()); + log.debug("{} : Shutdown({}): Subscriber cancelled.", streamIdentifier, shardInfo.shardId()); } markForShutdown(ShutdownReason.LEASE_LOST); return isShutdown(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java index 4f1db733..0f18891c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java @@ -21,8 +21,9 @@ import lombok.experimental.Accessors; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; -import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; @@ -41,7 +42,7 @@ public class ShardConsumerArgument { @NonNull private final ShardInfo shardInfo; @NonNull - private final String streamName; + private final StreamIdentifier streamIdentifier; @NonNull private final LeaseCoordinator leaseCoordinator; @NonNull @@ -71,4 +72,5 @@ public class ShardConsumerArgument { private final HierarchicalShardSyncer hierarchicalShardSyncer; @NonNull private final MetricsFactory metricsFactory; + private final LeaseCleanupManager leaseCleanupManager; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java index cb70024d..685a76d2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java @@ -24,6 +24,7 @@ import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; +import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RecordsRetrieved; import software.amazon.kinesis.retrieval.RetryableRetrievalException; @@ -40,8 +41,8 @@ class ShardConsumerSubscriber implements Subscriber { private final int bufferSize; private final ShardConsumer shardConsumer; private final int readTimeoutsToIgnoreBeforeWarning; + private final String shardInfoId; private volatile int readTimeoutSinceLastRead = 0; - @VisibleForTesting final Object lockObject = new Object(); // This holds the last time an attempt of request to upstream service was made including the first try to @@ -70,6 +71,7 @@ class ShardConsumerSubscriber implements Subscriber { this.bufferSize = bufferSize; this.shardConsumer = shardConsumer; this.readTimeoutsToIgnoreBeforeWarning = readTimeoutsToIgnoreBeforeWarning; + this.shardInfoId = ShardInfo.getLeaseKey(shardConsumer.shardInfo()); } @@ -107,7 +109,7 @@ class ShardConsumerSubscriber implements Subscriber { if (retrievalFailure != null) { synchronized (lockObject) { String logMessage = String.format("%s: Failure occurred in retrieval. Restarting data requests", - shardConsumer.shardInfo().shardId()); + shardInfoId); if (retrievalFailure instanceof RetryableRetrievalException) { log.debug(logMessage, retrievalFailure.getCause()); } else { @@ -130,7 +132,7 @@ class ShardConsumerSubscriber implements Subscriber { if (timeSinceLastResponse.toMillis() > maxTimeBetweenRequests) { log.error( "{}: Last request was dispatched at {}, but no response as of {} ({}). Cancelling subscription, and restarting. Last successful request details -- {}", - shardConsumer.shardInfo().shardId(), lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails()); + shardInfoId, lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails()); cancel(); // Start the subscription again which will update the lastRequestTime as well. @@ -157,7 +159,7 @@ class ShardConsumerSubscriber implements Subscriber { subscription); } catch (Throwable t) { - log.warn("{}: Caught exception from handleInput", shardConsumer.shardInfo().shardId(), t); + log.warn("{}: Caught exception from handleInput", shardInfoId, t); synchronized (lockObject) { dispatchFailure = t; } @@ -192,8 +194,8 @@ class ShardConsumerSubscriber implements Subscriber { protected void logOnErrorWarning(Throwable t) { log.warn( "{}: onError(). Cancelling subscription, and marking self as failed. KCL will " - + "recreate the subscription as neccessary to continue processing. Last successful request details -- {}", - shardConsumer.shardInfo().shardId(), recordsPublisher.getLastSuccessfulRequestDetails(), t); + + "recreate the subscription as necessary to continue processing. Last successful request details -- {}", + shardInfoId, recordsPublisher.getLastSuccessfulRequestDetails(), t); } protected void logOnErrorReadTimeoutWarning(Throwable t) { @@ -202,14 +204,14 @@ class ShardConsumerSubscriber implements Subscriber { + "are seeing this warning frequently consider increasing the SDK timeouts " + "by providing an OverrideConfiguration to the kinesis client. Alternatively you" + "can configure LifecycleConfig.readTimeoutsToIgnoreBeforeWarning to suppress" - + "intermittant ReadTimeout warnings. Last successful request details -- {}", - shardConsumer.shardInfo().shardId(), recordsPublisher.getLastSuccessfulRequestDetails(), t); + + "intermittent ReadTimeout warnings. Last successful request details -- {}", + shardInfoId, recordsPublisher.getLastSuccessfulRequestDetails(), t); } @Override public void onComplete() { log.debug("{}: onComplete(): Received onComplete. Activity should be triggered externally", - shardConsumer.shardInfo().shardId()); + shardInfoId); } public void cancel() { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 2bfcd358..c2c5c790 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -16,35 +16,46 @@ package software.amazon.kinesis.lifecycle; import com.google.common.annotations.VisibleForTesting; -import com.sun.org.apache.bcel.internal.generic.LUSHR; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; -import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; -import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; -import software.amazon.kinesis.leases.HierarchicalShardSyncer; +import software.amazon.kinesis.leases.UpdateField; +import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; import software.amazon.kinesis.metrics.MetricsFactory; -import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; /** * Task for invoking the ShardRecordProcessor shutdown() callback. @@ -55,6 +66,8 @@ import java.util.List; public class ShutdownTask implements ConsumerTask { private static final String SHUTDOWN_TASK_OPERATION = "ShutdownTask"; private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown"; + @VisibleForTesting + static final int RETRY_RANDOM_MAX_RANGE = 30; @NonNull private final ShardInfo shardInfo; @@ -82,10 +95,18 @@ public class ShutdownTask implements ConsumerTask { private final TaskType taskType = TaskType.SHUTDOWN; + private final List childShards; + @NonNull + private final StreamIdentifier streamIdentifier; + @NonNull + private final LeaseCleanupManager leaseCleanupManager; + + private static final Function leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo); + /* * Invokes ShardRecordProcessor shutdown() API. * (non-Javadoc) - * + * * @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerTask#call() */ @Override @@ -94,84 +115,48 @@ public class ShutdownTask implements ConsumerTask { final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHUTDOWN_TASK_OPERATION); Exception exception; - boolean applicationException = false; try { try { - 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.SHARD_END) { - latestShards = shardDetector.listShards(); + log.debug("Invoking shutdown() for shard {} with childShards {}, concurrencyToken {}. Shutdown reason: {}", + leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason); - //If latestShards is empty, should also shutdown the ShardConsumer without checkpoint with SHARD_END - if (CollectionUtils.isNullOrEmpty(latestShards) || !isShardInContextParentOfAny(latestShards)) { - localReason = ShutdownReason.LEASE_LOST; - dropLease(); - log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfo.shardId()); - } - } - - // If we reached end of the shard, set sequence number to SHARD_END. - if (localReason == ShutdownReason.SHARD_END) { - recordProcessorCheckpointer - .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); - recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); - } - - log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", - shardInfo.shardId(), shardInfo.concurrencyToken(), localReason); - final ShutdownInput shutdownInput = ShutdownInput.builder().shutdownReason(localReason) - .checkpointer(recordProcessorCheckpointer).build(); final long startTime = System.currentTimeMillis(); - try { - if (localReason == ShutdownReason.SHARD_END) { - shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); - if (lastCheckpointValue == null - || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { - throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfo.shardId() + ". Application must checkpoint upon shard end. " + - "See ShardRecordProcessor.shardEnded javadocs for more information."); - } - } else { - shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()); + final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + final Runnable leaseLostAction = () -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()); + + if (reason == ShutdownReason.SHARD_END) { + try { + takeShardEndAction(currentShardLease, scope, startTime); + } catch (InvalidStateException e) { + // If InvalidStateException happens, it indicates we have a non recoverable error in short term. + // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry shutting down. + log.warn("Lease {}: Invalid state encountered while shutting down shardConsumer with SHARD_END reason. " + + "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. ", leaseKeyProvider.apply(shardInfo), e); + dropLease(currentShardLease); + throwOnApplicationException(leaseLostAction, scope, startTime); } - log.debug("Shutting down retrieval strategy."); - recordsPublisher.shutdown(); - log.debug("Record processor completed shutdown() for shard {}", shardInfo.shardId()); - } catch (Exception e) { - applicationException = true; - throw e; - } finally { - MetricsUtil.addLatency(scope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); + } else { + throwOnApplicationException(leaseLostAction, scope, startTime); } - if (localReason == ShutdownReason.SHARD_END) { - log.debug("Looking for child shards of shard {}", shardInfo.shardId()); - // create leases for the child shards - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); - log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); - } + log.debug("Shutting down retrieval strategy for shard {}.", leaseKeyProvider.apply(shardInfo)); + recordsPublisher.shutdown(); + log.debug("Record processor completed shutdown() for shard {}", leaseKeyProvider.apply(shardInfo)); return new TaskResult(null); } catch (Exception e) { - if (applicationException) { - log.error("Application exception. ", e); + if (e instanceof CustomerApplicationException) { + log.error("Shard {}: Application exception. ", leaseKeyProvider.apply(shardInfo), e); } else { - log.error("Caught exception: ", e); + log.error("Shard {}: Caught exception: ", leaseKeyProvider.apply(shardInfo), e); } exception = e; // backoff if we encounter an exception. try { Thread.sleep(this.backoffTimeMillis); } catch (InterruptedException ie) { - log.debug("Interrupted sleep", ie); + log.debug("Shard {}: Interrupted sleep", leaseKeyProvider.apply(shardInfo), ie); } } } finally { @@ -179,12 +164,142 @@ public class ShutdownTask implements ConsumerTask { } return new TaskResult(exception); + } + // Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup. + private void takeShardEndAction(Lease currentShardLease, + MetricsScope scope, long startTime) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, + CustomerApplicationException { + // 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(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. + // In this case, KinesisDataFetcher and FanOutRecordsPublisher 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. + if (currentShardLease == null) { + throw new InvalidStateException(leaseKeyProvider.apply(shardInfo) + + " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner."); + } + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateLeaseWithChildShards(currentShardLease); + } + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, + shardInfo, shardDetector); + if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { + boolean isSuccess = false; + try { + isSuccess = attemptShardEndCheckpointing(scope, startTime); + } finally { + // Check if either the shard end ddb persist is successful or + // if childshards is empty. When child shards is empty then either it is due to + // completed shard being reprocessed or we got RNF from service. + // For these cases enqueue the lease for deletion. + if (isSuccess || CollectionUtils.isNullOrEmpty(childShards)) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + } + } + } + } + + private boolean attemptShardEndCheckpointing(MetricsScope scope, long startTime) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, + CustomerApplicationException { + final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) + .orElseThrow(() -> new InvalidStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); + if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { + // Call the shardRecordsProcessor to checkpoint with SHARD_END sequence number. + // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded. + throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime); + } + return true; + } + + private void applicationCheckpointAndVerification() { + recordProcessorCheckpointer + .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); + recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); + shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); + if (lastCheckpointValue == null + || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { + throw new IllegalArgumentException("Application didn't checkpoint at end of shard " + + leaseKeyProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + + "See ShardRecordProcessor.shardEnded javadocs for more information."); + } + } + + private void throwOnApplicationException(Runnable action, MetricsScope metricsScope, final long startTime) throws CustomerApplicationException { + try { + action.run(); + } catch (Exception e) { + throw new CustomerApplicationException("Customer application throws exception for shard " + leaseKeyProvider.apply(shardInfo) +": ", e); + } finally { + MetricsUtil.addLatency(metricsScope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); + } + } + + private void createLeasesForChildShardsIfNotExist() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + // For child shard resulted from merge of two parent shards, verify if both the parents are either present or + // not present in the lease table before creating the lease entry. + if (!CollectionUtils.isNullOrEmpty(childShards) && childShards.size() == 1) { + final ChildShard childShard = childShards.get(0); + final List parentLeaseKeys = childShard.parentShards().stream() + .map(parentShardId -> ShardInfo.getLeaseKey(shardInfo, parentShardId)).collect(Collectors.toList()); + if (parentLeaseKeys.size() != 2) { + throw new InvalidStateException("Shard " + shardInfo.shardId() + "'s only child shard " + childShard + + " does not contain other parent information."); + } else { + boolean isValidLeaseTableState = + Objects.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(0))) == Objects + .isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(1))); + if (!isValidLeaseTableState) { + if (!isOneInNProbability(RETRY_RANDOM_MAX_RANGE)) { + throw new BlockedOnParentShardException( + "Shard " + shardInfo.shardId() + "'s only child shard " + childShard + + " has partial parent information in lease table. Hence deferring lease creation of child shard."); + } else { + throw new InvalidStateException( + "Shard " + shardInfo.shardId() + "'s only child shard " + childShard + + " has partial parent information in lease table. Hence deferring lease creation of child shard."); + } + } + } + } + // Attempt create leases for child shards. + for(ChildShard childShard : childShards) { + final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId()); + if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) { + final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); + leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); + log.info("Shard {}: Created child shard lease: {}", shardInfo.shardId(), leaseToCreate.leaseKey()); + } + } + } + + /** + * Returns true for 1 in N probability. + */ + @VisibleForTesting + boolean isOneInNProbability(int n) { + Random r = new Random(); + return 1 == r.nextInt((n - 1) + 1) + 1; + } + + private void updateLeaseWithChildShards(Lease currentLease) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + Set childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet()); + + final Lease updatedLease = currentLease.copy(); + updatedLease.childShardIds(childShardIds); + leaseCoordinator.leaseRefresher().updateLeaseWithMetaInfo(updatedLease, UpdateField.CHILD_SHARDS); + log.info("Shard {}: Updated current lease {} with child shard information: {}", shardInfo.shardId(), currentLease.leaseKey(), childShardIds); } /* * (non-Javadoc) - * + * * @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerTask#taskType() */ @Override @@ -197,26 +312,13 @@ public class ShutdownTask implements ConsumerTask { return reason; } - private boolean isShardInContextParentOfAny(List shards) { - for(Shard shard : shards) { - if (isChildShardOfShardInContext(shard)) { - return true; - } - } - return false; - } - - private boolean isChildShardOfShardInContext(Shard shard) { - return (StringUtils.equals(shard.parentShardId(), shardInfo.shardId()) - || StringUtils.equals(shard.adjacentParentShardId(), shardInfo.shardId())); - } - - private void dropLease() { - Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId()); - leaseCoordinator.dropLease(currentLease); - if(currentLease != null) { - log.warn("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); + private void dropLease(Lease currentLease) { + if (currentLease == null) { + log.warn("Shard {}: Unable to find the lease for shard. Will shutdown the shardConsumer directly.", leaseKeyProvider.apply(shardInfo)); + return; + } else { + leaseCoordinator.dropLease(currentLease); + log.info("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); } } - } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java index d6c586aa..3717a805 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java @@ -47,4 +47,12 @@ public class InitializationInput { * completing the checkpoint. */ private final ExtendedSequenceNumber pendingCheckpointSequenceNumber; + + /** + * The last pending application state of the previous record processor. May be null. + * + * This will only be set if the previous record processor had prepared a checkpoint, but lost its lease before + * completing the checkpoint. + */ + private final byte[] pendingCheckpointState; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java index b7dd4e05..1ce9239b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java @@ -23,6 +23,7 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.ToString; import lombok.experimental.Accessors; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.processor.RecordProcessorCheckpointer; import software.amazon.kinesis.retrieval.KinesisClientRecord; @@ -56,6 +57,7 @@ public class ProcessRecordsInput { * The records received from Kinesis. These records may have been de-aggregated if they were published by the KPL. */ private List records; + /** * A checkpointer that the {@link ShardRecordProcessor} can use to checkpoint its progress. */ @@ -66,6 +68,11 @@ public class ProcessRecordsInput { * This value does not include the {@link #timeSpentInCache()}. */ private Long millisBehindLatest; + /** + * A list of child shards if the current GetRecords request reached the shard end. + * If not at the shard end, this should be an empty list. + */ + private List childShards; /** * How long the records spent waiting to be dispatched to the {@link ShardRecordProcessor} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java index c6f2fe6f..20c7c244 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java @@ -19,6 +19,7 @@ import org.apache.commons.lang3.StringUtils; import lombok.NonNull; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; +import software.amazon.kinesis.common.StreamIdentifier; /** * @@ -26,6 +27,7 @@ import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; public class MetricsUtil { public static final String OPERATION_DIMENSION_NAME = "Operation"; public static final String SHARD_ID_DIMENSION_NAME = "ShardId"; + public static final String STREAM_IDENTIFIER = "StreamId"; private static final String WORKER_IDENTIFIER_DIMENSION = "WorkerIdentifier"; private static final String TIME_METRIC = "Time"; private static final String SUCCESS_METRIC = "Success"; @@ -51,6 +53,11 @@ public class MetricsUtil { addOperation(metricsScope, SHARD_ID_DIMENSION_NAME, shardId); } + public static void addStreamId(@NonNull final MetricsScope metricsScope, @NonNull final StreamIdentifier streamId) { + streamId.accountIdOptional() + .ifPresent(acc -> addOperation(metricsScope, STREAM_IDENTIFIER, streamId.serialize())); + } + public static void addWorkerIdentifier(@NonNull final MetricsScope metricsScope, @NonNull final String workerIdentifier) { addOperation(metricsScope, WORKER_IDENTIFIER_DIMENSION, workerIdentifier); @@ -87,6 +94,11 @@ public class MetricsUtil { metricsScope.addData(metricName, success ? 1 : 0, StandardUnit.COUNT, metricsLevel); } + public static void addCount(@NonNull final MetricsScope metricsScope, final String dimension, + final long count, @NonNull final MetricsLevel metricsLevel) { + metricsScope.addData(dimension, count, StandardUnit.COUNT, metricsLevel); + } + public static void endScope(@NonNull final MetricsScope metricsScope) { metricsScope.end(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java index d3ecebc1..2ffadc06 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java @@ -27,52 +27,68 @@ public interface Checkpointer { * Record a checkpoint for a shard (e.g. sequence and subsequence numbers of last record processed * by application). Upon failover, record processing is resumed from this point. * - * @param shardId Checkpoint is specified for this shard. + * @param leaseKey Checkpoint is specified for this shard. * @param checkpointValue Value of the checkpoint (e.g. Kinesis sequence number and subsequence number) * @param concurrencyToken Used with conditional writes to prevent stale updates * (e.g. if there was a fail over to a different record processor, we don't want to * overwrite it's checkpoint) * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint */ - void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken) + void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken) throws KinesisClientLibException; /** * Get the current checkpoint stored for the specified shard. Useful for checking that the parent shard * has been completely processed before we start processing the child shard. - * - * @param shardId Current checkpoint for this shard is fetched + * + * @param leaseKey Current checkpoint for this shard is fetched * @return Current checkpoint for this shard, null if there is no record for this shard. * @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint */ - ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException; + ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException; /** * Get the current checkpoint stored for the specified shard, which holds the sequence numbers for the checkpoint * and pending checkpoint. Useful for checking that the parent shard has been completely processed before we start * processing the child shard. * - * @param shardId Current checkpoint for this shard is fetched + * @param leaseKey Current checkpoint for this shard is fetched * @return Current checkpoint object for this shard, null if there is no record for this shard. * @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint */ - Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException; + Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException; /** * Record intent to checkpoint for a shard. Upon failover, the pendingCheckpointValue will be passed to the new * ShardRecordProcessor's initialize() method. * - * @param shardId Checkpoint is specified for this shard. + * @param leaseKey Checkpoint is specified for this shard. * @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number) * @param concurrencyToken Used with conditional writes to prevent stale updates * (e.g. if there was a fail over to a different record processor, we don't want to * overwrite it's checkpoint) * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint */ - void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) + void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException; + /** + * Record intent to checkpoint for a shard. Upon failover, the pendingCheckpoint and pendingCheckpointState will be + * passed to the new ShardRecordProcessor's initialize() method. + * + * @param leaseKey Checkpoint is specified for this shard. + * @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number) + * @param concurrencyToken Used with conditional writes to prevent stale updates + * (e.g. if there was a fail over to a different record processor, we don't want to + * overwrite it's checkpoint) + * @param pendingCheckpointState Serialized application state at the pending checkpoint. + * + * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint + */ + void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) + throws KinesisClientLibException; + void operation(String operation); String operation(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java new file mode 100644 index 00000000..232c428d --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java @@ -0,0 +1,109 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.processor; + +import software.amazon.kinesis.common.StreamIdentifier; + +import java.time.Duration; +import java.util.List; + +/** + * Strategy for cleaning up the leases for former streams. + */ +public interface FormerStreamsLeasesDeletionStrategy { + + /** + * StreamIdentifiers for which leases needs to be cleaned up in the lease table. + * @return + */ + List streamIdentifiersForLeaseCleanup(); + + /** + * Duration to wait before deleting the leases for this stream. + * @return + */ + Duration waitPeriodToDeleteFormerStreams(); + + /** + * Strategy type for deleting the leases of former active streams. + * @return + */ + StreamsLeasesDeletionType leaseDeletionType(); + + /** + * StreamsLeasesDeletionType identifying the different lease cleanup strategies. + */ + enum StreamsLeasesDeletionType { + NO_STREAMS_LEASES_DELETION, + FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION, + PROVIDED_STREAMS_DEFERRED_DELETION + } + + /** + * Strategy for not cleaning up leases for former streams. + */ + final class NoLeaseDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final List streamIdentifiersForLeaseCleanup() { + throw new UnsupportedOperationException("StreamIdentifiers not required"); + } + + @Override + public final Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION; + } + } + + /** + * Strategy for auto detection the old of former streams based on the {@link MultiStreamTracker#streamConfigList()} + * and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()} + */ + abstract class AutoDetectionAndDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final List streamIdentifiersForLeaseCleanup() { + throw new UnsupportedOperationException("StreamIdentifiers not required"); + } + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION; + } + } + + /** + * Strategy to detect the streams for deletion through {@link #streamIdentifiersForLeaseCleanup()} provided by customer at runtime + * and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()} + */ + abstract class ProvidedStreamsDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION; + } + } + +} + + + + diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java new file mode 100644 index 00000000..1b742509 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java @@ -0,0 +1,45 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.processor; + +import software.amazon.kinesis.common.StreamConfig; + +import java.util.List; + +/** + * Interface for stream trackers. This is useful for KCL Workers that need + * to consume data from multiple streams. + * KCL will periodically probe this interface to learn about the new and old streams. + */ +public interface MultiStreamTracker { + + /** + * Returns the list of stream config, to be processed by the current application. + * Note that the streams list CAN be changed during the application runtime. + * This method will be called periodically by the KCL to learn about the change in streams to process. + * + * @return List of StreamConfig + */ + List streamConfigList(); + + /** + * Strategy to delete leases of old streams in the lease table. + * Note that the strategy CANNOT be changed during the application runtime. + * + * @return StreamsLeasesDeletionStrategy + */ + FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy(); +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java index 2eb3f5c1..34b2930c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java @@ -93,7 +93,6 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; - /** * This method will checkpoint the progress at the provided sequenceNumber and subSequenceNumber, the latter for * aggregated records produced with the Producer Library. This method is analogous to {@link #checkpoint()} @@ -145,6 +144,32 @@ public interface RecordProcessorCheckpointer { PreparedCheckpointer prepareCheckpoint() throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** + * This method will record a pending checkpoint at the last data record that was delivered to the record processor. + * If the application fails over between calling prepareCheckpoint() and checkpoint(), the init() method of the next + * IRecordProcessor for this shard will be informed of the prepared sequence number and application state. + * + * Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having + * side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete. + * Use the sequence number and application state passed in to init() to behave idempotently. + * + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + */ + PreparedCheckpointer prepareCheckpoint(byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** * This method will record a pending checkpoint at the at the provided record. This method is analogous to * {@link #prepareCheckpoint()} but provides the ability to specify the record at which to prepare the checkpoint. @@ -174,6 +199,38 @@ public interface RecordProcessorCheckpointer { PreparedCheckpointer prepareCheckpoint(Record record) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** + * This method will record a pending checkpoint at the at the provided record. This method is analogous to + * {@link #prepareCheckpoint()} but provides the ability to specify the record and application state at which to + * prepare the checkpoint. + * + * @param record A record at which to prepare checkpoint in this shard. + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * + * Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having + * side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete. + * Use the sequence number and application state passed in to init() to behave idempotently. + * + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ + PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** * This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to * {@link #prepareCheckpoint()} but provides the ability to specify the sequence number at which to checkpoint. @@ -200,6 +257,35 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + /** + * This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to + * {@link #prepareCheckpoint()} but provides the ability to specify the sequence number and application state + * at which to checkpoint. + * + * @param sequenceNumber A sequence number at which to prepare checkpoint in this shard. + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ + PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; + /** * This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for * aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()} @@ -228,5 +314,36 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + /** + * This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for + * aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()} + * but provides the ability to specify the sequence number, subsequence number, and application state at which to + * checkpoint. + * + * @param sequenceNumber A sequence number at which to prepare checkpoint in this shard. + * @param subSequenceNumber A subsequence number at which to prepare checkpoint within this shard. + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ + PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; + Checkpointer checkpointer(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java index b0559120..72a6d66f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java @@ -15,6 +15,8 @@ package software.amazon.kinesis.processor; +import software.amazon.kinesis.common.StreamIdentifier; + /** * */ @@ -25,4 +27,13 @@ public interface ShardRecordProcessorFactory { * @return */ ShardRecordProcessor shardRecordProcessor(); + + /** + * Returns a new instance of the ShardRecordProcessor for a stream identifier + * @param streamIdentifier + * @return ShardRecordProcessor + */ + default ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { + return shardRecordProcessor(); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java new file mode 100644 index 00000000..b5c7b23e --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.retrieval; + +import java.time.Duration; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.metrics.MetricsFactory; + +public interface DataFetcherProviderConfig { + + /** + * Gets stream identifier for dataFetcher. + */ + StreamIdentifier getStreamIdentifier(); + + /** + * Gets shard id. + */ + String getShardId(); + + /** + * Gets current instance of metrics factory. + */ + MetricsFactory getMetricsFactory(); + + /** + * Gets current max records allowed to process at a given time. + */ + Integer getMaxRecords(); + + /** + * Gets timeout for kinesis request. + */ + Duration getKinesisRequestTimeout(); +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java new file mode 100644 index 00000000..ba743e61 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.retrieval; + +import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.utils.CollectionUtils; + +import java.util.List; + +public class DataRetrievalUtil { + + public static boolean isValidResult(String shardEndIndicator, List childShards) { + // shardEndIndicator is nextShardIterator for GetRecordsResponse, and is continuationSequenceNumber for SubscribeToShardEvent + // There are two valid scenarios for the shardEndIndicator and childShards combination. + // 1. ShardEnd scenario: shardEndIndicator should be null and childShards should be a non-empty list. + // 2. Non-ShardEnd scenario: shardEndIndicator should be non-null and childShards should be null or an empty list. + // Otherwise, the retrieval result is invalid. + if (shardEndIndicator == null && CollectionUtils.isNullOrEmpty(childShards) || + shardEndIndicator != null && !CollectionUtils.isNullOrEmpty(childShards)) { + return false; + } + + // For ShardEnd scenario, for each childShard we should validate if parentShards are available. + // Missing parentShards can cause issues with creating leases for childShards during ShardConsumer shutdown. + if (!CollectionUtils.isNullOrEmpty(childShards)) { + for (ChildShard childShard : childShards) { + if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + return false; + } + } + } + return true; + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java index ca0487f3..3ff8e620 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java @@ -14,7 +14,9 @@ */ package software.amazon.kinesis.retrieval; +import java.util.Optional; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.kinesis.retrieval.polling.DataFetcher; import software.amazon.kinesis.retrieval.polling.KinesisDataFetcher; /** @@ -41,15 +43,33 @@ public interface GetRecordsRetrievalStrategy { /** * Returns whether this strategy has been shutdown. - * + * * @return true if the strategy has been shutdown, false otherwise. */ boolean isShutdown(); /** - * Returns the KinesisDataFetcher used to records from Kinesis. - * - * @return KinesisDataFetcher + * Returns a DataFetcher used to records from Kinesis. + * + * @return DataFetcher */ KinesisDataFetcher getDataFetcher(); + + /** + * Returns a DataFetcher override if applicable, else empty for retrieving records from Kinesis. + * + * @return Optional + */ + default Optional getDataFetcherOverride() { + return Optional.empty(); + } + + /** + * Returns a dataFetcher by first checking for an override if it exists, else using the default data fetcher. + * + * @return DataFetcher + */ + default DataFetcher dataFetcher() { + return getDataFetcherOverride().orElse(getDataFetcher()); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java new file mode 100644 index 00000000..7cf6cdcf --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.retrieval; + +import java.time.Duration; +import lombok.Data; +import lombok.NonNull; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.metrics.MetricsFactory; + + +/** + * Configuration needed for custom data fetchers + */ +@Data +public class KinesisDataFetcherProviderConfig implements DataFetcherProviderConfig { + + @NonNull + private StreamIdentifier streamIdentifier; + + @NonNull + private String shardId; + + @NonNull + private MetricsFactory metricsFactory; + + @NonNull + private Integer maxRecords; + + @NonNull + private Duration kinesisRequestTimeout; +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 9ca73aa5..e3ef9d63 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -15,18 +15,29 @@ package software.amazon.kinesis.retrieval; -import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.Getter; import lombok.NonNull; +import lombok.Setter; +import lombok.ToString; import lombok.experimental.Accessors; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.fanout.FanOutConfig; +import software.amazon.kinesis.retrieval.polling.PollingConfig; /** * Used by the KCL to configure the retrieval of records from Kinesis. */ -@Data +@Getter +@Setter +@ToString +@EqualsAndHashCode @Accessors(fluent = true) public class RetrievalConfig { /** @@ -34,7 +45,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.2.12-SNAPSHOT"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.3.0"; /** * Client used to make calls to Kinesis for records retrieval @@ -42,15 +53,15 @@ public class RetrievalConfig { @NonNull private final KinesisAsyncClient kinesisClient; - /** - * The name of the stream to process records from. - */ - @NonNull - private final String streamName; - @NonNull private final String applicationName; + + /** + * AppStreamTracker either for multi stream tracking or single stream + */ + private Either appStreamTracker; + /** * Backoff time between consecutive ListShards calls. * @@ -84,15 +95,77 @@ public class RetrievalConfig { private RetrievalFactory retrievalFactory; - public RetrievalFactory retrievalFactory() { + public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName, + @NonNull String applicationName) { + this.kinesisClient = kinesisAsyncClient; + this.appStreamTracker = Either + .right(new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStreamExtended)); + this.applicationName = applicationName; + } + public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull MultiStreamTracker multiStreamTracker, + @NonNull String applicationName) { + this.kinesisClient = kinesisAsyncClient; + this.appStreamTracker = Either.left(multiStreamTracker); + this.applicationName = applicationName; + } + + public RetrievalConfig initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) { + final StreamConfig[] streamConfig = new StreamConfig[1]; + this.appStreamTracker.apply(multiStreamTracker -> { + throw new IllegalArgumentException( + "Cannot set initialPositionInStreamExtended when multiStreamTracker is set"); + }, sc -> streamConfig[0] = sc); + this.appStreamTracker = Either + .right(new StreamConfig(streamConfig[0].streamIdentifier(), initialPositionInStreamExtended)); + return this; + } + + public RetrievalConfig retrievalSpecificConfig(RetrievalSpecificConfig retrievalSpecificConfig) { + this.retrievalSpecificConfig = retrievalSpecificConfig; + validateFanoutConfig(); + validatePollingConfig(); + return this; + } + + public RetrievalFactory retrievalFactory() { if (retrievalFactory == null) { if (retrievalSpecificConfig == null) { - retrievalSpecificConfig = new FanOutConfig(kinesisClient()).streamName(streamName()) + retrievalSpecificConfig = new FanOutConfig(kinesisClient()) .applicationName(applicationName()); + retrievalSpecificConfig = appStreamTracker.map(multiStreamTracker -> retrievalSpecificConfig, + streamConfig -> ((FanOutConfig) retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName())); } retrievalFactory = retrievalSpecificConfig.retrievalFactory(); } return retrievalFactory; } + + private void validateFanoutConfig() { + // If we are in multistream mode and if retrievalSpecificConfig is an instance of FanOutConfig and if consumerArn is set throw exception. + boolean isFanoutConfig = retrievalSpecificConfig instanceof FanOutConfig; + boolean isInvalidFanoutConfig = isFanoutConfig && appStreamTracker.map( + multiStreamTracker -> ((FanOutConfig) retrievalSpecificConfig).consumerArn() != null + || ((FanOutConfig) retrievalSpecificConfig).streamName() != null, + streamConfig -> streamConfig.streamIdentifier() == null + || streamConfig.streamIdentifier().streamName() == null); + if(isInvalidFanoutConfig) { + throw new IllegalArgumentException( + "Invalid config: Either in multi-stream mode with streamName/consumerArn configured or in single-stream mode with no streamName configured"); + } + } + + private void validatePollingConfig() { + boolean isPollingConfig = retrievalSpecificConfig instanceof PollingConfig; + boolean isInvalidPollingConfig = isPollingConfig && appStreamTracker.map( + multiStreamTracker -> + ((PollingConfig) retrievalSpecificConfig).streamName() != null, + streamConfig -> + streamConfig.streamIdentifier() == null || streamConfig.streamIdentifier().streamName() == null); + + if (isInvalidPollingConfig) { + throw new IllegalArgumentException( + "Invalid config: Either in multi-stream mode with streamName configured or in single-stream mode with no streamName configured"); + } + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java index 4c8f6b68..5703e1af 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java @@ -15,6 +15,7 @@ package software.amazon.kinesis.retrieval; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; @@ -24,5 +25,10 @@ import software.amazon.kinesis.metrics.MetricsFactory; public interface RetrievalFactory { GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(ShardInfo shardInfo, MetricsFactory metricsFactory); + @Deprecated RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory); + + default RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, StreamConfig streamConfig, MetricsFactory metricsFactory) { + return createGetRecordsCache(shardInfo, metricsFactory); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java index 5ab982bf..30562994 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java @@ -15,10 +15,13 @@ package software.amazon.kinesis.retrieval; +import java.util.function.Function; +import software.amazon.kinesis.retrieval.polling.DataFetcher; + public interface RetrievalSpecificConfig { /** * Creates and returns a retrieval factory for the specific configuration - * + * * @return a retrieval factory that can create an appropriate retriever */ RetrievalFactory retrievalFactory(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java index 661c2841..9318b996 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java @@ -80,17 +80,12 @@ public class FanOutConfig implements RetrievalSpecificConfig { */ private long retryBackoffMillis = 1000; - @Override - public RetrievalFactory retrievalFactory() { - return new FanOutRetrievalFactory(kinesisClient, getOrCreateConsumerArn()); + @Override public RetrievalFactory retrievalFactory() { + return new FanOutRetrievalFactory(kinesisClient, streamName, consumerArn, this::getOrCreateConsumerArn); } - private String getOrCreateConsumerArn() { - if (consumerArn != null) { - return consumerArn; - } - - FanOutConsumerRegistration registration = createConsumerRegistration(); + private String getOrCreateConsumerArn(String streamName) { + FanOutConsumerRegistration registration = createConsumerRegistration(streamName); try { return registration.getOrCreateStreamConsumerArn(); } catch (DependencyException e) { @@ -98,10 +93,10 @@ public class FanOutConfig implements RetrievalSpecificConfig { } } - private FanOutConsumerRegistration createConsumerRegistration() { + private FanOutConsumerRegistration createConsumerRegistration(String streamName) { String consumerToCreate = ObjectUtils.firstNonNull(consumerName(), applicationName()); return createConsumerRegistration(kinesisClient(), - Preconditions.checkNotNull(streamName(), "streamName must be set for consumer creation"), + Preconditions.checkNotNull(streamName, "streamName must be set for consumer creation"), Preconditions.checkNotNull(consumerToCreate, "applicationName or consumerName must be set for consumer creation")); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java index 0519390c..9bcdd83c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java @@ -76,7 +76,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { try { response = describeStreamConsumer(); } catch (ResourceNotFoundException e) { - log.info("StreamConsumer not found, need to create it."); + log.info("{} : StreamConsumer not found, need to create it.", streamName); } // 2. If not, register consumer @@ -92,7 +92,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { break; } catch (LimitExceededException e) { // TODO: Figure out internal service exceptions - log.debug("RegisterStreamConsumer call got throttled will retry."); + log.debug("{} : RegisterStreamConsumer call got throttled will retry.", streamName); finalException = e; } retries--; @@ -104,7 +104,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { } } catch (ResourceInUseException e) { // Consumer is present, call DescribeStreamConsumer - log.debug("Got ResourceInUseException consumer exists, will call DescribeStreamConsumer again."); + log.debug("{} : Got ResourceInUseException consumer exists, will call DescribeStreamConsumer again.", streamName); response = describeStreamConsumer(); } } @@ -160,17 +160,17 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { while (!ConsumerStatus.ACTIVE.equals(status) && retries > 0) { status = describeStreamConsumer().consumerDescription().consumerStatus(); retries--; - log.info(String.format("Waiting for StreamConsumer %s to have ACTIVE status...", streamConsumerName)); + log.info("{} : Waiting for StreamConsumer {} to have ACTIVE status...", streamName, streamConsumerName); Thread.sleep(retryBackoffMillis); } } catch (InterruptedException ie) { - log.debug("Thread was interrupted while fetching StreamConsumer status, moving on."); + log.debug("{} : Thread was interrupted while fetching StreamConsumer status, moving on.", streamName); } if (!ConsumerStatus.ACTIVE.equals(status)) { final String message = String.format( - "Status of StreamConsumer %s, was not ACTIVE after all retries. Was instead %s.", - streamConsumerName, status); + "%s : Status of StreamConsumer %s, was not ACTIVE after all retries. Was instead %s.", + streamName, streamConsumerName, status); log.error(message); throw new IllegalStateException(message); } @@ -211,7 +211,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { throw new DependencyException(e); } } catch (LimitExceededException e) { - log.info("Throttled while calling {} API, will backoff.", apiName); + log.info("{} : Throttled while calling {} API, will backoff.", streamName, apiName); try { Thread.sleep(retryBackoffMillis + (long) (Math.random() * 100)); } catch (InterruptedException ie) { @@ -224,7 +224,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { if (finalException == null) { throw new IllegalStateException( - String.format("Finished all retries and no exception was caught while calling %s", apiName)); + String.format("%s : Finished all retries and no exception was caught while calling %s", streamName, apiName)); } throw finalException; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index 27cad136..7e8932cf 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java @@ -27,17 +27,20 @@ import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; import software.amazon.awssdk.core.async.SdkPublisher; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponse; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.KinesisRequestsBuilder; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.retrieval.BatchUniqueIdentifier; import software.amazon.kinesis.retrieval.IteratorBuilder; @@ -51,7 +54,6 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.time.Instant; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -59,8 +61,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired; +import static software.amazon.kinesis.retrieval.DataRetrievalUtil.isValidResult; -@RequiredArgsConstructor @Slf4j @KinesisClientInternalApi public class FanOutRecordsPublisher implements RecordsPublisher { @@ -73,7 +75,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private final KinesisAsyncClient kinesis; private final String shardId; private final String consumerArn; - + private final String streamAndShardId; private final Object lockObject = new Object(); private final AtomicInteger subscribeToShardId = new AtomicInteger(0); @@ -91,11 +93,25 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private RequestDetails lastSuccessfulRequestDetails = new RequestDetails(); + public FanOutRecordsPublisher(KinesisAsyncClient kinesis, String shardId, String consumerArn) { + this.kinesis = kinesis; + this.shardId = shardId; + this.consumerArn = consumerArn; + this.streamAndShardId = shardId; + } + + public FanOutRecordsPublisher(KinesisAsyncClient kinesis, String shardId, String consumerArn, String streamIdentifierSer) { + this.kinesis = kinesis; + this.shardId = shardId; + this.consumerArn = consumerArn; + this.streamAndShardId = streamIdentifierSer + ":" + shardId; + } + @Override public void start(ExtendedSequenceNumber extendedSequenceNumber, InitialPositionInStreamExtended initialPositionInStreamExtended) { synchronized (lockObject) { - log.debug("[{}] Initializing Publisher @ Sequence: {} -- Initial Position: {}", shardId, + log.debug("[{}] Initializing Publisher @ Sequence: {} -- Initial Position: {}", streamAndShardId, extendedSequenceNumber, initialPositionInStreamExtended); this.initialPositionInStreamExtended = initialPositionInStreamExtended; this.currentSequenceNumber = extendedSequenceNumber.sequenceNumber(); @@ -174,7 +190,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // It is now safe to remove the element recordsDeliveryQueue.poll(); // Take action based on the time spent by the event in queue. - takeDelayedDeliveryActionIfRequired(shardId, recordsRetrievedContext.getEnqueueTimestamp(), log); + takeDelayedDeliveryActionIfRequired(streamAndShardId, recordsRetrievedContext.getEnqueueTimestamp(), log); // Update current sequence number for the successfully delivered event. currentSequenceNumber = ((FanoutRecordsRetrieved)recordsRetrieved).continuationSequenceNumber(); // Update the triggering flow for post scheduling upstream request. @@ -190,13 +206,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow != null && recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier() .equals(flow.getSubscribeToShardId())) { log.error( - "{}: Received unexpected ack for the active subscription {}. Throwing. ", - shardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); + "{}: Received unexpected ack for the active subscription {}. Throwing. ", streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); throw new IllegalStateException("Unexpected ack for the active subscription"); } // Otherwise publisher received a stale ack. else { - log.info("{}: Publisher received an ack for stale subscription {}. Ignoring.", shardId, + log.info("{}: Publisher received an ack for stale subscription {}. Ignoring.", streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); } } @@ -219,10 +234,10 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (IllegalStateException e) { log.warn("{}: Unable to enqueue the payload due to capacity restrictions in delivery queue with remaining capacity {}. Last successful request details -- {}", - shardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails); + streamAndShardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails); throw e; } catch (Throwable t) { - log.error("{}: Unable to deliver event to the shard consumer.", shardId, t); + log.error("{}: Unable to deliver event to the shard consumer.", streamAndShardId, t); throw t; } } @@ -290,7 +305,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { String instanceId = shardId + "-" + subscribeInvocationId; log.debug( "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#subscribeToShard) @ {} id: {} -- Starting subscribe to shard", - shardId, connectionStart, instanceId); + streamAndShardId, connectionStart, instanceId); flow = new RecordFlow(this, connectionStart, instanceId); kinesis.subscribeToShard(request, flow); } @@ -303,12 +318,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if(hasValidFlow()) { log.warn( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- Subscriber is null." + - " Last successful request details -- {}", shardId, flow.connectionStartedAt, + " Last successful request details -- {}", streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId, lastSuccessfulRequestDetails); } else { log.warn( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) -- Subscriber and flow are null." + - " Last successful request details -- {}", shardId, lastSuccessfulRequestDetails); + " Last successful request details -- {}", streamAndShardId, lastSuccessfulRequestDetails); } return; } @@ -320,8 +335,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow != null) { String logMessage = String.format( "%s: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ %s id: %s -- %s." + - " Last successful request details -- %s", - shardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails); + " Last successful request details -- %s", streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails); switch (category.throwableType) { case READ_TIMEOUT: log.debug(logMessage, propagationThrowable); @@ -339,13 +353,14 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } flow.cancel(); } - log.debug("{}: availableQueueSpace zeroing from {}", shardId, availableQueueSpace); + log.debug("{}: availableQueueSpace zeroing from {}", streamAndShardId, availableQueueSpace); availableQueueSpace = 0; try { handleFlowError(propagationThrowable, triggeringFlow); } catch (Throwable innerThrowable) { - log.warn("{}: Exception while calling subscriber.onError. Last successful request details -- {}", shardId, lastSuccessfulRequestDetails, innerThrowable); + log.warn("{}: Exception while calling subscriber.onError. Last successful request details -- {}", + streamAndShardId, lastSuccessfulRequestDetails, innerThrowable); } subscriber = null; flow = null; @@ -353,7 +368,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (triggeringFlow != null) { log.debug( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- {} -> triggeringFlow wasn't the active flow. Didn't dispatch error", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId, + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId, category.throwableTypeString); triggeringFlow.cancel(); } @@ -367,7 +382,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // Clear any lingering records in the queue. if (!recordsDeliveryQueue.isEmpty()) { log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of" - + "previous subscription - {}. Last successful request details -- {}", shardId, subscribeToShardId, lastSuccessfulRequestDetails); + + "previous subscription - {}. Last successful request details -- {}", streamAndShardId, subscribeToShardId, lastSuccessfulRequestDetails); recordsDeliveryQueue.clear(); } } @@ -383,11 +398,11 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (t.getCause() instanceof ResourceNotFoundException) { log.debug( "{}: Could not call SubscribeToShard successfully because shard no longer exists. Marking shard for completion.", - shardId); + streamAndShardId); // The ack received for this onNext event will be ignored by the publisher as the global flow object should // be either null or renewed when the ack's flow identifier is evaluated. FanoutRecordsRetrieved response = new FanoutRecordsRetrieved( - ProcessRecordsInput.builder().records(Collections.emptyList()).isAtShardEnd(true).build(), null, + ProcessRecordsInput.builder().records(Collections.emptyList()).isAtShardEnd(true).childShards(Collections.emptyList()).build(), null, triggeringFlow != null ? triggeringFlow.getSubscribeToShardId() : shardId + "-no-flow-found"); subscriber.onNext(response); subscriber.onComplete(); @@ -452,7 +467,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (!hasValidSubscriber()) { log.debug( "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Subscriber is null.", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); triggeringFlow.cancel(); if (flow != null) { flow.cancel(); @@ -462,23 +477,36 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (!isActiveFlow(triggeringFlow)) { log.debug( "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Received records for an inactive flow.", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); return; } - List records = recordBatchEvent.records().stream().map(KinesisClientRecord::fromRecord) - .collect(Collectors.toList()); - ProcessRecordsInput input = ProcessRecordsInput.builder().cacheEntryTime(Instant.now()) - .millisBehindLatest(recordBatchEvent.millisBehindLatest()) - .isAtShardEnd(recordBatchEvent.continuationSequenceNumber() == null).records(records).build(); - FanoutRecordsRetrieved recordsRetrieved = new FanoutRecordsRetrieved(input, - recordBatchEvent.continuationSequenceNumber(), triggeringFlow.subscribeToShardId); - try { + // If recordBatchEvent is not valid event, RuntimeException will be thrown here and trigger the errorOccurred call. + // Since the triggeringFlow is active flow, it will then trigger the handleFlowError call. + // Since the exception is not ResourceNotFoundException, it will trigger onError in the ShardConsumerSubscriber. + // The ShardConsumerSubscriber will finally cancel the subscription. + if (!isValidResult(recordBatchEvent.continuationSequenceNumber(), recordBatchEvent.childShards())) { + throw new InvalidStateException("RecordBatchEvent for flow " + triggeringFlow.toString() + " is invalid." + + " event.continuationSequenceNumber: " + recordBatchEvent.continuationSequenceNumber() + + ". event.childShards: " + recordBatchEvent.childShards()); + } + + List records = recordBatchEvent.records().stream().map(KinesisClientRecord::fromRecord) + .collect(Collectors.toList()); + ProcessRecordsInput input = ProcessRecordsInput.builder() + .cacheEntryTime(Instant.now()) + .millisBehindLatest(recordBatchEvent.millisBehindLatest()) + .isAtShardEnd(recordBatchEvent.continuationSequenceNumber() == null) + .records(records) + .childShards(recordBatchEvent.childShards()) + .build(); + FanoutRecordsRetrieved recordsRetrieved = new FanoutRecordsRetrieved(input, + recordBatchEvent.continuationSequenceNumber(), triggeringFlow.subscribeToShardId); bufferCurrentEventAndScheduleIfRequired(recordsRetrieved, triggeringFlow); } catch (Throwable t) { log.warn("{}: Unable to buffer or schedule onNext for subscriber. Failing publisher." + - " Last successful request details -- {}", shardId, lastSuccessfulRequestDetails); + " Last successful request details -- {}", streamAndShardId, lastSuccessfulRequestDetails); errorOccurred(triggeringFlow, t); } } @@ -488,7 +516,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (availableQueueSpace <= 0) { log.debug( "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Attempted to decrement availableQueueSpace to below 0", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); } else { availableQueueSpace--; if (availableQueueSpace > 0) { @@ -503,12 +531,13 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private void onComplete(RecordFlow triggeringFlow) { synchronized (lockObject) { - log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", shardId, + log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); triggeringFlow.cancel(); if (!hasValidSubscriber()) { - log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", shardId, + log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); return; } @@ -516,15 +545,15 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (!isActiveFlow(triggeringFlow)) { log.debug( "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {} -- Received spurious onComplete from unexpected flow. Ignoring.", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); return; } if (currentSequenceNumber != null) { - log.debug("{}: Shard hasn't ended. Resubscribing.", shardId); + log.debug("{}: Shard hasn't ended. Resubscribing.", streamAndShardId); subscribeToShard(currentSequenceNumber); } else { - log.debug("{}: Shard has ended completing subscriber.", shardId); + log.debug("{}: Shard has ended completing subscriber.", streamAndShardId); subscriber.onComplete(); } } @@ -536,7 +565,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (subscriber != null) { log.error( "{}: A subscribe occurred while there was an active subscriber. Sending error to current subscriber", - shardId); + streamAndShardId); MultipleSubscriberException multipleSubscriberException = new MultipleSubscriberException(); // @@ -575,7 +604,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (subscriber != s) { log.warn( "{}: (FanOutRecordsPublisher/Subscription#request) - Rejected an attempt to request({}), because subscribers don't match. Last successful request details -- {}", - shardId, n, lastSuccessfulRequestDetails); + streamAndShardId, n, lastSuccessfulRequestDetails); return; } if (flow == null) { @@ -584,7 +613,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // log.debug( "{}: (FanOutRecordsPublisher/Subscription#request) - Request called for a null flow.", - shardId); + streamAndShardId); errorOccurred(flow, new IllegalStateException("Attempted to request on a null flow.")); return; } @@ -602,19 +631,19 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (subscriber != s) { log.warn( "{}: (FanOutRecordsPublisher/Subscription#cancel) - Rejected attempt to cancel subscription, because subscribers don't match. Last successful request details -- {}", - shardId, lastSuccessfulRequestDetails); + streamAndShardId, lastSuccessfulRequestDetails); return; } if (!hasValidSubscriber()) { log.warn( "{}: (FanOutRecordsPublisher/Subscription#cancel) - Cancelled called even with an invalid subscriber. Last successful request details -- {}", - shardId, lastSuccessfulRequestDetails); + streamAndShardId, lastSuccessfulRequestDetails); } subscriber = null; if (flow != null) { log.debug( "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher/Subscription#cancel) @ {} id: {}", - shardId, flow.connectionStartedAt, flow.subscribeToShardId); + streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId); flow.cancel(); availableQueueSpace = 0; } @@ -703,12 +732,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { public void onEventStream(SdkPublisher publisher) { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- Subscribe", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); if (!parent.isActiveFlow(this)) { this.isDisposed = true; log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- parent is disposed", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); parent.rejectSubscription(publisher); return; } @@ -716,7 +745,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { try { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- creating record subscription", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); subscription = new RecordSubscription(parent, this, connectionStartedAt, subscribeToShardId); publisher.subscribe(subscription); @@ -727,7 +756,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (Throwable t) { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- throwable during record subscription: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, t.getMessage()); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage()); parent.errorOccurred(this, t); } } @@ -736,7 +765,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { @Override public void responseReceived(SubscribeToShardResponse response) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#responseReceived) @ {} id: {} -- Response received. Request id - {}", - parent.shardId, connectionStartedAt, subscribeToShardId, response.responseMetadata().requestId()); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, response.responseMetadata().requestId()); final RequestDetails requestDetails = new RequestDetails(response.responseMetadata().requestId(), connectionStartedAt.toString()); parent.setLastSuccessfulRequestDetails(requestDetails); @@ -759,12 +788,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- {}: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); if (this.isDisposed) { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- This flow has been disposed, not dispatching error. {}: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); this.isErrorDispatched = true; } @@ -775,7 +804,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } else { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- An error has previously been dispatched, not dispatching this error {}: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); } } @@ -802,7 +831,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (Exception e) { log.warn( "{}: Unable to enqueue the {} shutdown event due to capacity restrictions in delivery queue with remaining capacity {}. Ignoring. Last successful request details -- {}", - parent.shardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(), + parent.streamAndShardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(), parent.lastSuccessfulRequestDetails, subscriptionShutdownEvent.getShutdownEventThrowableOptional()); } } @@ -810,7 +839,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private void executeComplete() { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Connection completed", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); if (isCancelled) { // @@ -820,13 +849,13 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // subscription, which was cancelled for a reason (usually queue overflow). // log.warn("{}: complete called on a cancelled subscription. Ignoring completion. Last successful request details -- {}", - parent.shardId, parent.lastSuccessfulRequestDetails); + parent.streamAndShardId, parent.lastSuccessfulRequestDetails); return; } if (this.isDisposed) { log.warn( "{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- This flow has been disposed not dispatching completion. Last successful request details -- {}", - parent.shardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails); return; } @@ -844,7 +873,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (Throwable t) { log.error( "{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Exception while trying to cancel failed subscription: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t); } } } @@ -885,14 +914,14 @@ public class FanOutRecordsPublisher implements RecordsPublisher { public void cancel() { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- Cancel called", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); flow.isCancelled = true; if (subscription != null) { subscription.cancel(); } else { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- SDK subscription is null", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } } } @@ -906,21 +935,21 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow.isCancelled) { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Subscription was cancelled before onSubscribe", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } if (flow.isDisposed) { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow has been disposed cancelling subscribe", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow requires cancelling", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); cancel(); } log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Outstanding: {} items so requesting an item", - parent.shardId, connectionStartedAt, subscribeToShardId, parent.availableQueueSpace); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.availableQueueSpace); if (parent.availableQueueSpace > 0) { request(1); } @@ -933,7 +962,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow.shouldSubscriptionCancel()) { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onNext) @ {} id: {} -- RecordFlow requires cancelling", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); cancel(); return; } @@ -948,7 +977,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { @Override public void onError(Throwable t) { - log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#onError) @ {} id: {} -- {}: {}", parent.shardId, + log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#onError) @ {} id: {} -- {}: {}", parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getClass().getName(), t.getMessage()); // @@ -961,7 +990,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { public void onComplete() { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onComplete) @ {} id: {} -- Allowing RecordFlow to call onComplete", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java index 4add0dab..35301624 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java @@ -19,18 +19,29 @@ import lombok.NonNull; import lombok.RequiredArgsConstructor; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + @RequiredArgsConstructor @KinesisClientInternalApi public class FanOutRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; - private final String consumerArn; + private final String defaultStreamName; + private final String defaultConsumerArn; + private final Function consumerArnCreator; + + private Map implicitConsumerArnTracker = new HashMap<>(); @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo, @@ -40,7 +51,28 @@ public class FanOutRetrievalFactory implements RetrievalFactory { @Override public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, + final StreamConfig streamConfig, final MetricsFactory metricsFactory) { - return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), consumerArn); + final Optional streamIdentifierStr = shardInfo.streamIdentifierSerOpt(); + if(streamIdentifierStr.isPresent()) { + final StreamIdentifier streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()); + return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), + getOrCreateConsumerArn(streamIdentifier, streamConfig.consumerArn()), + streamIdentifierStr.get()); + } else { + final StreamIdentifier streamIdentifier = StreamIdentifier.singleStreamInstance(defaultStreamName); + return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), + getOrCreateConsumerArn(streamIdentifier, defaultConsumerArn)); + } + } + + @Override + public RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory) { + throw new UnsupportedOperationException("FanoutRetrievalFactory needs StreamConfig Info"); + } + + private String getOrCreateConsumerArn(StreamIdentifier streamIdentifier, String consumerArn) { + return consumerArn != null ? consumerArn : implicitConsumerArnTracker + .computeIfAbsent(streamIdentifier, sId -> consumerArnCreator.apply(sId.streamName())); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java index e2de9786..0c1c4a28 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java @@ -141,6 +141,11 @@ public class ExtendedSequenceNumber implements Comparable dataFetcherProvider; /** * Name of the Kinesis stream. * * @return String */ - @NonNull - private final String streamName; + private String streamName; + + /** + * @param kinesisClient Client used to access Kinesis services. + */ + public PollingConfig(KinesisAsyncClient kinesisClient) { + this.kinesisClient = kinesisClient; + } /** * Client used to access to Kinesis service. @@ -60,6 +76,15 @@ public class PollingConfig implements RetrievalSpecificConfig { */ private int maxRecords = 10000; + /** + * @param streamName Name of Kinesis stream. + * @param kinesisClient Client used to access Kinesis serivces. + */ + public PollingConfig(String streamName, KinesisAsyncClient kinesisClient) { + this.kinesisClient = kinesisClient; + this.streamName = streamName; + } + /** * The value for how long the ShardConsumer should sleep if no records are returned from the call to * {@link KinesisAsyncClient#getRecords(GetRecordsRequest)}. @@ -105,6 +130,6 @@ public class PollingConfig implements RetrievalSpecificConfig { @Override public RetrievalFactory retrievalFactory() { return new SynchronousBlockingRetrievalFactory(streamName(), kinesisClient(), recordsFetcherFactory, - maxRecords(), kinesisRequestTimeout); + maxRecords(), kinesisRequestTimeout, dataFetcherProvider); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index dcd5e043..ef752f1b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -15,6 +15,8 @@ package software.amazon.kinesis.retrieval.polling; +import com.google.common.annotations.VisibleForTesting; + import java.time.Duration; import java.time.Instant; import java.util.List; @@ -25,21 +27,17 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; - import lombok.AccessLevel; +import lombok.Data; import lombok.Getter; +import lombok.NonNull; import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.Validate; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; - -import com.google.common.annotations.VisibleForTesting; - -import lombok.Data; -import lombok.NonNull; -import lombok.experimental.Accessors; -import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; @@ -47,6 +45,7 @@ import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsLevel; @@ -61,7 +60,6 @@ import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RecordsRetrieved; import software.amazon.kinesis.retrieval.RetryableRetrievalException; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; - import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired; /** @@ -91,7 +89,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { private final DefaultGetRecordsCacheDaemon defaultGetRecordsCacheDaemon; private boolean started = false; private final String operation; - private final String shardId; + private final StreamIdentifier streamId; + private final String streamAndShardId; private Subscriber subscriber; @VisibleForTesting @Getter private final PublisherSession publisherSession; @@ -108,7 +107,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { @VisibleForTesting @Getter private final LinkedBlockingQueue prefetchRecordsQueue; private final PrefetchCounters prefetchCounters; - private final KinesisDataFetcher dataFetcher; + private final DataFetcher dataFetcher; private InitialPositionInStreamExtended initialPositionInStreamExtended; private String highestSequenceNumber; @@ -135,11 +134,11 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { // Handle records delivery ack and execute nextEventDispatchAction. // This method is not thread-safe and needs to be called after acquiring a monitor. - void handleRecordsDeliveryAck(RecordsDeliveryAck recordsDeliveryAck, String shardId, Runnable nextEventDispatchAction) { + void handleRecordsDeliveryAck(RecordsDeliveryAck recordsDeliveryAck, String streamAndShardId, Runnable nextEventDispatchAction) { final PrefetchRecordsRetrieved recordsToCheck = peekNextRecord(); // Verify if the ack matches the head of the queue and evict it. if (recordsToCheck != null && recordsToCheck.batchUniqueIdentifier().equals(recordsDeliveryAck.batchUniqueIdentifier())) { - evictPublishedRecordAndUpdateDemand(shardId); + evictPublishedRecordAndUpdateDemand(streamAndShardId); nextEventDispatchAction.run(); } else { // Log and ignore any other ack received. As long as an ack is received for head of the queue @@ -148,21 +147,21 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { final BatchUniqueIdentifier peekedBatchUniqueIdentifier = recordsToCheck == null ? null : recordsToCheck.batchUniqueIdentifier(); log.info("{} : Received a stale notification with id {} instead of expected id {} at {}. Will ignore.", - shardId, recordsDeliveryAck.batchUniqueIdentifier(), peekedBatchUniqueIdentifier, Instant.now()); + streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier(), peekedBatchUniqueIdentifier, Instant.now()); } } // Evict the published record from the prefetch queue. // This method is not thread-safe and needs to be called after acquiring a monitor. @VisibleForTesting - RecordsRetrieved evictPublishedRecordAndUpdateDemand(String shardId) { + RecordsRetrieved evictPublishedRecordAndUpdateDemand(String streamAndShardId) { final PrefetchRecordsRetrieved result = prefetchRecordsQueue.poll(); if (result != null) { updateDemandTrackersOnPublish(result); } else { log.info( "{}: No record batch found while evicting from the prefetch queue. This indicates the prefetch buffer" - + "was reset.", shardId); + + " was reset.", streamAndShardId); } return result; } @@ -215,14 +214,15 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { this.maxByteSize = maxByteSize; this.maxRecordsCount = maxRecordsCount; this.publisherSession = new PublisherSession(new LinkedBlockingQueue<>(this.maxPendingProcessRecordsInput), - new PrefetchCounters(), this.getRecordsRetrievalStrategy.getDataFetcher()); + new PrefetchCounters(), this.getRecordsRetrievalStrategy.dataFetcher()); this.executorService = executorService; this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory); this.idleMillisBetweenCalls = idleMillisBetweenCalls; this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon(); Validate.notEmpty(operation, "Operation cannot be empty"); this.operation = operation; - this.shardId = shardId; + this.streamId = this.getRecordsRetrievalStrategy.dataFetcher().getStreamIdentifier(); + this.streamAndShardId = this.streamId.serialize() + ":" + shardId; } @Override @@ -230,12 +230,12 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { if (executorService.isShutdown()) { throw new IllegalStateException("ExecutorService has been shutdown."); } - - publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended); - if (!started) { - log.info("{} : Starting prefetching thread.", shardId); + log.info("{} : Starting Prefetching thread and initializing publisher session.", streamAndShardId); + publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended); executorService.execute(defaultGetRecordsCacheDaemon); + } else { + log.info("{} : Skipping publisher start as it was already started.", streamAndShardId); } started = true; } @@ -304,9 +304,9 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { @Override public synchronized void notify(RecordsDeliveryAck recordsDeliveryAck) { - publisherSession.handleRecordsDeliveryAck(recordsDeliveryAck, shardId, () -> drainQueueForRequests()); + publisherSession.handleRecordsDeliveryAck(recordsDeliveryAck, streamAndShardId, () -> drainQueueForRequests()); // Take action based on the time spent by the event in queue. - takeDelayedDeliveryActionIfRequired(shardId, lastEventDeliveryTime, log); + takeDelayedDeliveryActionIfRequired(streamAndShardId, lastEventDeliveryTime, log); } // Note : Do not make this method synchronous as notify() will not be able to evict any entry from the queue. @@ -403,7 +403,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { public void run() { while (!isShutdown) { if (Thread.currentThread().isInterrupted()) { - log.warn("{} : Prefetch thread was interrupted.", shardId); + log.warn("{} : Prefetch thread was interrupted.", streamAndShardId); break; } @@ -411,7 +411,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { try { makeRetrievalAttempt(); } catch(PositionResetException pre) { - log.debug("{} : Position was reset while attempting to add item to queue.", shardId); + log.debug("{} : Position was reset while attempting to add item to queue.", streamAndShardId); } finally { resetLock.readLock().unlock(); } @@ -435,7 +435,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { .records(records) .millisBehindLatest(getRecordsResult.millisBehindLatest()) .cacheEntryTime(lastSuccessfulCall) - .isAtShardEnd(getRecordsRetrievalStrategy.getDataFetcher().isShardEndReached()) + .isAtShardEnd(getRecordsRetrievalStrategy.dataFetcher().isShardEndReached()) + .childShards(getRecordsResult.childShards()) .build(); PrefetchRecordsRetrieved recordsRetrieved = new PrefetchRecordsRetrieved(processRecordsInput, @@ -447,23 +448,24 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { } catch (PositionResetException pse) { throw pse; } catch (RetryableRetrievalException rre) { - log.info("{} : Timeout occurred while waiting for response from Kinesis. Will retry the request.", shardId); + log.info("{} : Timeout occurred while waiting for response from Kinesis. Will retry the request.", streamAndShardId); } catch (InterruptedException e) { - log.info("{} : Thread was interrupted, indicating shutdown was called on the cache.", shardId); + log.info("{} : Thread was interrupted, indicating shutdown was called on the cache.", streamAndShardId); } catch (ExpiredIteratorException e) { log.info("{} : records threw ExpiredIteratorException - restarting" - + " after greatest seqNum passed to customer", shardId, e); + + " after greatest seqNum passed to customer", streamAndShardId, e); + MetricsUtil.addStreamId(scope, streamId); scope.addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.COUNT, MetricsLevel.SUMMARY); publisherSession.dataFetcher().restartIterator(); } catch (SdkException e) { - log.error("{} : Exception thrown while fetching records from Kinesis", shardId, e); + log.error("{} : Exception thrown while fetching records from Kinesis", streamAndShardId, e); } catch (Throwable e) { log.error("{} : Unexpected exception was thrown. This could probably be an issue or a bug." + " Please search for the exception/error online to check what is going on. If the " + "issue persists or is a recurring problem, feel free to open an issue on, " + - "https://github.com/awslabs/amazon-kinesis-client.", shardId, e); + "https://github.com/awslabs/amazon-kinesis-client.", streamAndShardId, e); } finally { MetricsUtil.endScope(scope); } @@ -475,7 +477,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { publisherSession.prefetchCounters().waitForConsumer(); } catch (InterruptedException ie) { log.info("{} : Thread was interrupted while waiting for the consumer. " + - "Shutdown has probably been started", shardId); + "Shutdown has probably been started", streamAndShardId); } } } @@ -522,14 +524,14 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { public synchronized void waitForConsumer() throws InterruptedException { if (!shouldGetNewRecords()) { - log.debug("{} : Queue is full waiting for consumer for {} ms", shardId, idleMillisBetweenCalls); + log.debug("{} : Queue is full waiting for consumer for {} ms", streamAndShardId, idleMillisBetweenCalls); this.wait(idleMillisBetweenCalls); } } public synchronized boolean shouldGetNewRecords() { if (log.isDebugEnabled()) { - log.debug("{} : Current Prefetch Counter States: {}", shardId, this.toString()); + log.debug("{} : Current Prefetch Counter States: {}", streamAndShardId, this.toString()); } return size < maxRecordsCount && byteSize < maxByteSize; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java index ac40c7d2..071763fc 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java @@ -15,19 +15,22 @@ package software.amazon.kinesis.retrieval.polling; +import java.time.Duration; +import java.util.function.Function; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.retrieval.DataFetcherProviderConfig; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; +import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RecordsFetcherFactory; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; -import java.time.Duration; - /** * */ @@ -41,29 +44,71 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; @NonNull private final RecordsFetcherFactory recordsFetcherFactory; - // private final long listShardsBackoffTimeInMillis; - // private final int maxListShardsRetryAttempts; + private final int maxRecords; private final Duration kinesisRequestTimeout; - public SynchronousBlockingRetrievalFactory(String streamName, KinesisAsyncClient kinesisClient, RecordsFetcherFactory recordsFetcherFactory, int maxRecords, Duration kinesisRequestTimeout) { + private final Function dataFetcherProvider; + + @Deprecated + public SynchronousBlockingRetrievalFactory(String streamName, + KinesisAsyncClient kinesisClient, + RecordsFetcherFactory recordsFetcherFactory, + int maxRecords, + Duration kinesisRequestTimeout) { + this(streamName, + kinesisClient, + recordsFetcherFactory, + maxRecords, + kinesisRequestTimeout, + defaultDataFetcherProvider(kinesisClient)); + } + + public SynchronousBlockingRetrievalFactory(String streamName, + KinesisAsyncClient kinesisClient, + RecordsFetcherFactory recordsFetcherFactory, + int maxRecords, + Duration kinesisRequestTimeout, + Function dataFetcherProvider) { this.streamName = streamName; this.kinesisClient = kinesisClient; this.recordsFetcherFactory = recordsFetcherFactory; this.maxRecords = maxRecords; this.kinesisRequestTimeout = kinesisRequestTimeout; + this.dataFetcherProvider = dataFetcherProvider == null ? + defaultDataFetcherProvider(kinesisClient) : dataFetcherProvider; } @Deprecated - public SynchronousBlockingRetrievalFactory(String streamName, KinesisAsyncClient kinesisClient, RecordsFetcherFactory recordsFetcherFactory, int maxRecords) { + public SynchronousBlockingRetrievalFactory(String streamName, + KinesisAsyncClient kinesisClient, + RecordsFetcherFactory recordsFetcherFactory, + int maxRecords) { this(streamName, kinesisClient, recordsFetcherFactory, maxRecords, PollingConfig.DEFAULT_REQUEST_TIMEOUT); } + private static Function defaultDataFetcherProvider( + KinesisAsyncClient kinesisClient) { + return dataFetcherProviderConfig -> new KinesisDataFetcher(kinesisClient, dataFetcherProviderConfig); + } + @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, - @NonNull final MetricsFactory metricsFactory) { - return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, streamName, shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); + @NonNull final MetricsFactory metricsFactory) { + final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ? + StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) : + StreamIdentifier.singleStreamInstance(streamName); + + final DataFetcherProviderConfig kinesisDataFetcherProviderConfig = new KinesisDataFetcherProviderConfig( + streamIdentifier, + shardInfo.shardId(), + metricsFactory, + maxRecords, + kinesisRequestTimeout); + + final DataFetcher dataFetcher = this.dataFetcherProvider.apply(kinesisDataFetcherProviderConfig); + + return new SynchronousGetRecordsRetrievalStrategy(dataFetcher); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java index c6fa619b..7f3b54d5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java @@ -14,6 +14,7 @@ */ package software.amazon.kinesis.retrieval.polling; +import java.util.Optional; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; @@ -26,8 +27,9 @@ import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @Data @KinesisClientInternalApi public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrievalStrategy { + @NonNull - private final KinesisDataFetcher dataFetcher; + private final DataFetcher dataFetcher; @Override public GetRecordsResponse getRecords(final int maxRecords) { @@ -45,9 +47,14 @@ public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetriev public boolean isShutdown() { return false; } - + @Override public KinesisDataFetcher getDataFetcher() { + throw new UnsupportedOperationException("Deprecated. Use dataFetcher() to retrieve a dataFetcher"); + } + + @Override + public DataFetcher dataFetcher() { return dataFetcher; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java index 320fe4dd..efa11e70 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java @@ -17,13 +17,14 @@ package software.amazon.kinesis.retrieval.polling; import java.time.Duration; import java.util.concurrent.ExecutorService; - import lombok.NonNull; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; +import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RecordsFetcherFactory; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; @@ -65,11 +66,20 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory this.maxFutureWait = maxFutureWait; } - @Override - public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, + @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { - return new SynchronousGetRecordsRetrievalStrategy(new KinesisDataFetcher(kinesisClient, streamName, - shardInfo.shardId(), maxRecords, metricsFactory, maxFutureWait)); + final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ? + StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) : + StreamIdentifier.singleStreamInstance(streamName); + + return new SynchronousGetRecordsRetrievalStrategy( + new KinesisDataFetcher(kinesisClient, new KinesisDataFetcherProviderConfig( + streamIdentifier, + shardInfo.shardId(), + metricsFactory, + maxRecords, + maxFutureWait + ))); } @Override diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java index 1cf77a3d..b823c8e3 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java @@ -90,6 +90,26 @@ public class CheckpointerTest { Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); } + + @Test + public final void testInitialPrepareCheckpointWithApplicationState() throws Exception { + String sequenceNumber = "1"; + String pendingCheckpointValue = "99999"; + String shardId = "myShardId"; + byte[] applicationState = "applicationState".getBytes(); + ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(sequenceNumber); + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken); + + ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue); + checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), testConcurrencyToken, + applicationState); + + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + } + @Test public final void testAdvancingPrepareCheckpoint() throws Exception { String shardId = "myShardId"; @@ -107,6 +127,26 @@ public class CheckpointerTest { } } + @Test + public final void testAdvancingPrepareCheckpointWithApplicationState() throws Exception { + String shardId = "myShardId"; + String checkpointValue = "12345"; + byte[] applicationState = "applicationState".getBytes(); + ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue); + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), testConcurrencyToken); + + for (Integer i = 0; i < 10; i++) { + String sequenceNumber = i.toString(); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber); + checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken, + applicationState); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + } + } + @Test public final void testPrepareAndSetCheckpoint() throws Exception { String checkpointValue = "12345"; @@ -134,4 +174,35 @@ public class CheckpointerTest { Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); } + + @Test + public final void testPrepareAndSetCheckpointWithApplicationState() throws Exception { + String checkpointValue = "12345"; + String shardId = "testShardId-1"; + String concurrencyToken = "token-1"; + String pendingCheckpointValue = "99999"; + byte[] applicationState = "applicationState".getBytes(); + + // set initial checkpoint + ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue); + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), concurrencyToken); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + + // prepare checkpoint + ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue); + checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken, applicationState); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + + // do checkpoint + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java index b8de6a1b..8f6e165d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java @@ -18,7 +18,6 @@ import java.util.HashMap; import java.util.Map; import software.amazon.kinesis.exceptions.KinesisClientLibException; -import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.processor.Checkpointer; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @@ -32,6 +31,7 @@ public class InMemoryCheckpointer implements Checkpointer { private Map checkpoints = new HashMap<>(); private Map flushpoints = new HashMap<>(); private Map pendingCheckpoints = new HashMap<>(); + private Map pendingCheckpointStates = new HashMap<>(); private String operation; @@ -39,14 +39,15 @@ public class InMemoryCheckpointer implements Checkpointer { * {@inheritDoc} */ @Override - public void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken) + public void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken) throws KinesisClientLibException { - checkpoints.put(shardId, checkpointValue); - flushpoints.put(shardId, checkpointValue); - pendingCheckpoints.remove(shardId); + checkpoints.put(leaseKey, checkpointValue); + flushpoints.put(leaseKey, checkpointValue); + pendingCheckpoints.remove(leaseKey); + pendingCheckpointStates.remove(leaseKey); if (log.isDebugEnabled()) { - log.debug("shardId: {} checkpoint: {}", shardId, checkpointValue); + log.debug("shardId: {} checkpoint: {}", leaseKey, checkpointValue); } } @@ -55,25 +56,32 @@ public class InMemoryCheckpointer implements Checkpointer { * {@inheritDoc} */ @Override - public ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException { - ExtendedSequenceNumber checkpoint = flushpoints.get(shardId); - log.debug("checkpoint shardId: {} checkpoint: {}", shardId, checkpoint); + public ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException { + ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); + log.debug("checkpoint shardId: {} checkpoint: {}", leaseKey, checkpoint); return checkpoint; } @Override - public void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException { - pendingCheckpoints.put(shardId, pendingCheckpoint); + prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null); } @Override - public Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException { - ExtendedSequenceNumber checkpoint = flushpoints.get(shardId); - ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(shardId); + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException { + pendingCheckpoints.put(leaseKey, pendingCheckpoint); + pendingCheckpointStates.put(leaseKey, pendingCheckpointState); + } - Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint); - log.debug("getCheckpointObject shardId: {}, {}", shardId, checkpointObj); + @Override + public Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException { + ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); + ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(leaseKey); + byte[] pendingCheckpointState = pendingCheckpointStates.get(leaseKey); + + Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint, pendingCheckpointState); + log.debug("getCheckpointObject shardId: {}, {}", leaseKey, checkpointObj); return checkpointObj; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java new file mode 100644 index 00000000..b300f355 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java @@ -0,0 +1,136 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.booleanThat; +import static org.mockito.Mockito.when; +import static software.amazon.kinesis.coordinator.DeterministicShuffleShardSyncLeaderDecider.DETERMINISTIC_SHUFFLE_SEED; + +@RunWith(MockitoJUnitRunner.class) +public class DeterministicShuffleShardSyncLeaderDeciderTest { + private static final String LEASE_KEY = "lease_key"; + private static final String LEASE_OWNER = "lease_owner"; + private static final String WORKER_ID = "worker-id"; + + private DeterministicShuffleShardSyncLeaderDecider leaderDecider; + + @Mock + private LeaseRefresher leaseRefresher; + + @Mock + private ScheduledExecutorService scheduledExecutorService; + + private int numShardSyncWorkers; + + @Before + public void setup() { + numShardSyncWorkers = 1; + leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers); + } + + @Test + public void testLeaderElectionWithNullLeases() { + boolean isLeader = leaderDecider.isLeader(WORKER_ID); + assertTrue("IsLeader should return true if leaders is null", isLeader); + } + + @Test + public void testLeaderElectionWithEmptyLeases() throws Exception { + when(leaseRefresher.listLeases()).thenReturn(new ArrayList<>()); + boolean isLeader = leaderDecider.isLeader(WORKER_ID); + assertTrue("IsLeader should return true if no leases are returned", isLeader); + } + + @Test + public void testleaderElectionWithEmptyOwnerLeases() throws Exception { + List leases = getLeases(5, true, true, true); + when(leaseRefresher.listLeases()).thenReturn(leases); + boolean isLeader = leaderDecider.isLeader(WORKER_ID); + assertTrue("IsLeader should return true if leases have no owner", isLeader); + } + + @Test + public void testElectedLeadersAsPerExpectedShufflingOrder() + throws Exception { + List leases = getLeases(5, false /*emptyLeaseOwner */,false /* duplicateLeaseOwner */, true /* activeLeases */); + when(leaseRefresher.listLeases()).thenReturn(leases); + Set expectedLeaders = getExpectedLeaders(leases); + for (String leader : expectedLeaders) { + assertTrue(leaderDecider.isLeader(leader)); + } + for (Lease lease : leases) { + if (!expectedLeaders.contains(lease.leaseOwner())) { + assertFalse(leaderDecider.isLeader(lease.leaseOwner())); + } + } + } + + @Test + public void testElectedLeadersAsPerExpectedShufflingOrderWhenUniqueWorkersLessThanMaxLeaders() { + this.numShardSyncWorkers = 5; // More than number of unique lease owners + leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers); + List leases = getLeases(3, false /*emptyLeaseOwner */, false /* duplicateLeaseOwner */, true /* activeLeases */); + Set expectedLeaders = getExpectedLeaders(leases); + // All lease owners should be present in expected leaders set, and they should all be leaders. + for (Lease lease : leases) { + assertTrue(leaderDecider.isLeader(lease.leaseOwner())); + assertTrue(expectedLeaders.contains(lease.leaseOwner())); + } + } + + private List getLeases(int count, boolean emptyLeaseOwner, boolean duplicateLeaseOwner, boolean activeLeases) { + List leases = new ArrayList<>(); + for (int i = 0; i < count; i++) { + Lease lease = new Lease(); + lease.leaseKey(LEASE_KEY + i); + lease.checkpoint(activeLeases ? ExtendedSequenceNumber.LATEST : ExtendedSequenceNumber.SHARD_END); + lease.leaseCounter(new Random().nextLong()); + if (!emptyLeaseOwner) { + lease.leaseOwner(LEASE_OWNER + (duplicateLeaseOwner ? "" : i)); + } + leases.add(lease); + } + return leases; + } + + private Set getExpectedLeaders(List leases) { + List uniqueHosts = leases.stream().filter(lease -> lease.leaseOwner() != null) + .map(Lease::leaseOwner).distinct().sorted().collect(Collectors.toList()); + + Collections.shuffle(uniqueHosts, new Random(DETERMINISTIC_SHUFFLE_SEED)); + int numWorkers = Math.min(uniqueHosts.size(), this.numShardSyncWorkers); + return new HashSet<>(uniqueHosts.subList(0, numWorkers)); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java new file mode 100644 index 00000000..a2047a6b --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java @@ -0,0 +1,589 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.coordinator; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.HashKeyRange; +import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.common.HashKeyRangeForLease; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.MultiStreamLease; +import software.amazon.kinesis.leases.ShardDetector; +import software.amazon.kinesis.leases.ShardSyncTaskManager; +import software.amazon.kinesis.metrics.NullMetricsFactory; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static software.amazon.kinesis.common.HashKeyRangeForLease.deserialize; +import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MAX_HASH_KEY; +import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MIN_HASH_KEY; +import static software.amazon.kinesis.leases.LeaseManagementConfig.DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY; + +@RunWith(MockitoJUnitRunner.class) + +public class PeriodicShardSyncManagerTest { + + private StreamIdentifier streamIdentifier; + private PeriodicShardSyncManager periodicShardSyncManager; + @Mock + private LeaderDecider leaderDecider; + @Mock + private LeaseRefresher leaseRefresher; + @Mock + Map currentStreamConfigMap; + @Mock + Function shardSyncTaskManagerProvider; + + @Before + public void setup() { + streamIdentifier = StreamIdentifier.multiStreamInstance("123:stream:456"); + periodicShardSyncManager = new PeriodicShardSyncManager("worker", leaderDecider, leaseRefresher, currentStreamConfigMap, + shardSyncTaskManagerProvider, true, new NullMetricsFactory(), 2 * 60 * 1000, 3); + } + + @Test + public void testForFailureWhenHashRangesAreIncomplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("25", MAX_HASH_KEY.toString())); // Missing interval here + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertTrue(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenHashRangesAreComplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenUnSortedHashRangesAreComplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("4", "23")); + add(deserialize("2", "3")); + add(deserialize("0", "1")); + add(deserialize("24", MAX_HASH_KEY.toString())); + add(deserialize("6", "23")); + + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenHashRangesAreCompleteForOverlappingLeasesAtEnd() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + add(deserialize("24", "45")); + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, null).shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenEmptyLeasesArePassed() { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, new ArrayList<>()).shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenConfidenceFactorIsNotReached() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + } + + @Test + public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsReached() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenHoleIsDueToShardEnd() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); // introducing hole here through SHARD_END checkpoint + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + if(lease.hashKeyRangeForLease().startingHashKey().toString().equals("4")) { + lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + } else { + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenNoLeasesAreUsedDueToShardEnd() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenHoleShifts() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + List multiStreamLeases2 = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); // Hole between 3 and 5 + add(deserialize("5", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + // Resetting the holes + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenHoleShiftsMoreThanOnce() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + List multiStreamLeases2 = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); // Hole between 3 and 5 + add(deserialize("5", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + // Resetting the holes + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync())); + // Resetting the holes + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); + } + + @Test + public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSync() { + ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); + ShardDetector shardDetector = mock(ShardDetector.class); + when(shardSyncTaskManagerProvider.apply(any())).thenReturn(shardSyncTaskManager); + when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + + final int[] shardCounter = { 0 }; + List hashKeyRangeForLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "20")); + add(deserialize("21", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}; + + List kinesisShards = hashKeyRangeForLeases.stream() + .map(hashKeyRangeForLease -> Shard.builder().shardId("shard-" + (++shardCounter[0])).hashKeyRange( + HashKeyRange.builder().startingHashKey(hashKeyRangeForLease.serializedStartingHashKey()) + .endingHashKey(hashKeyRangeForLease.serializedEndingHashKey()).build()).build()) + .collect(Collectors.toList()); + + when(shardDetector.listShards()).thenReturn(kinesisShards); + + final int[] leaseCounter = { 0 }; + List multiStreamLeases = hashKeyRangeForLeases.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0]))); + lease.shardId("shard-"+(leaseCounter[0])); + // Setting the hashrange only for last two leases + if(leaseCounter[0] >= 3) { + lease.hashKeyRange(hashKeyRangeForLease); + } + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + // Assert that shard sync should never trigger + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); + + // Assert that all the leases now has hashRanges set. + for(Lease lease : multiStreamLeases) { + Assert.assertNotNull(lease.hashKeyRangeForLease()); + } + } + + @Test + public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSyncInHoleScenario() { + ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); + ShardDetector shardDetector = mock(ShardDetector.class); + when(shardSyncTaskManagerProvider.apply(any())).thenReturn(shardSyncTaskManager); + when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + + final int[] shardCounter = { 0 }; + List hashKeyRangeForLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("5", "20")); // Hole between 3 and 5 + add(deserialize("21", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}; + + List kinesisShards = hashKeyRangeForLeases.stream() + .map(hashKeyRangeForLease -> Shard.builder().shardId("shard-" + (++shardCounter[0])).hashKeyRange( + HashKeyRange.builder().startingHashKey(hashKeyRangeForLease.serializedStartingHashKey()) + .endingHashKey(hashKeyRangeForLease.serializedEndingHashKey()).build()).build()) + .collect(Collectors.toList()); + + when(shardDetector.listShards()).thenReturn(kinesisShards); + + final int[] leaseCounter = { 0 }; + List multiStreamLeases = hashKeyRangeForLeases.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0]))); + lease.shardId("shard-"+(leaseCounter[0])); + // Setting the hashrange only for last two leases + if(leaseCounter[0] >= 3) { + lease.hashKeyRange(hashKeyRangeForLease); + } + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + // Assert that shard sync should never trigger + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); + + // Assert that all the leases now has hashRanges set. + for(Lease lease : multiStreamLeases) { + Assert.assertNotNull(lease.hashKeyRangeForLease()); + } + } + + @Test + public void testFor1000DifferentValidSplitHierarchyTreeTheHashRangesAreAlwaysComplete() { + for(int i=0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.SPLIT, maxInitialLeaseCount, false); + Collections.shuffle(leases); +// System.out.println( +// leases.stream().map(l -> l.checkpoint().sequenceNumber() + ":" + l.hashKeyRangeForLease()).collect(Collectors.toList())); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidMergeHierarchyTreeTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, false); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidReshardHierarchyTreeTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, false); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidMergeHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, true); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidReshardHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, true); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + + + private List generateInitialLeases(int initialShardCount) { + long hashRangeInternalMax = 10000000; + List initialLeases = new ArrayList<>(); + long leaseStartKey = 0; + for (int i = 1; i <= initialShardCount; i++) { + final Lease lease = new Lease(); + long leaseEndKey; + if (i != initialShardCount) { + leaseEndKey = (hashRangeInternalMax / initialShardCount) * i; + lease.hashKeyRange(HashKeyRangeForLease.deserialize(leaseStartKey + "", leaseEndKey + "")); + } else { + leaseEndKey = 0; + lease.hashKeyRange(HashKeyRangeForLease.deserialize(leaseStartKey + "", MAX_HASH_KEY.toString())); + } + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + lease.leaseKey("shard-" + i); + initialLeases.add(lease); + leaseStartKey = leaseEndKey + 1; + } + return initialLeases; + } + + private void reshard(List initialLeases, int depth, ReshardType reshardType, int leaseCounter, + boolean shouldKeepSomeParentsInProgress) { + for (int i = 0; i < depth; i++) { + if (reshardType == ReshardType.SPLIT) { + leaseCounter = split(initialLeases, leaseCounter); + } else if (reshardType == ReshardType.MERGE) { + leaseCounter = merge(initialLeases, leaseCounter, shouldKeepSomeParentsInProgress); + } else { + if (isHeads()) { + leaseCounter = split(initialLeases, leaseCounter); + } else { + leaseCounter = merge(initialLeases, leaseCounter, shouldKeepSomeParentsInProgress); + } + } + } + } + + private int merge(List initialLeases, int leaseCounter, boolean shouldKeepSomeParentsInProgress) { + List leasesEligibleForMerge = initialLeases.stream().filter(l -> CollectionUtils.isNullOrEmpty(l.childShardIds())) + .collect(Collectors.toList()); +// System.out.println("Leases to merge : " + leasesEligibleForMerge); + int leasesToMerge = (int) ((leasesEligibleForMerge.size() - 1) / 2.0 * Math.random()); + for (int i = 0; i < leasesToMerge; i += 2) { + Lease parent1 = leasesEligibleForMerge.get(i); + Lease parent2 = leasesEligibleForMerge.get(i + 1); + if(parent2.hashKeyRangeForLease().startingHashKey().subtract(parent1.hashKeyRangeForLease().endingHashKey()).equals(BigInteger.ONE)) + { + parent1.checkpoint(ExtendedSequenceNumber.SHARD_END); + if (!shouldKeepSomeParentsInProgress || (shouldKeepSomeParentsInProgress && isOneFromDiceRoll())) { +// System.out.println("Deciding to keep parent in progress : " + parent2); + parent2.checkpoint(ExtendedSequenceNumber.SHARD_END); + } + Lease child = new Lease(); + child.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child.leaseKey("shard-" + (++leaseCounter)); +// System.out.println("Parent " + parent1 + " and " + parent2 + " merges into " + child); + child.hashKeyRange(new HashKeyRangeForLease(parent1.hashKeyRangeForLease().startingHashKey(), + parent2.hashKeyRangeForLease().endingHashKey())); + parent1.childShardIds(Collections.singletonList(child.leaseKey())); + parent2.childShardIds(Collections.singletonList(child.leaseKey())); + child.parentShardIds(Sets.newHashSet(parent1.leaseKey(), parent2.leaseKey())); + + initialLeases.add(child); + } + } + return leaseCounter; + } + + private int split(List initialLeases, int leaseCounter) { + List leasesEligibleForSplit = initialLeases.stream().filter(l -> CollectionUtils.isNullOrEmpty(l.childShardIds())) + .collect(Collectors.toList()); +// System.out.println("Leases to split : " + leasesEligibleForSplit); + int leasesToSplit = (int) (leasesEligibleForSplit.size() * Math.random()); + for (int i = 0; i < leasesToSplit; i++) { + Lease parent = leasesEligibleForSplit.get(i); + parent.checkpoint(ExtendedSequenceNumber.SHARD_END); + Lease child1 = new Lease(); + child1.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child1.hashKeyRange(new HashKeyRangeForLease(parent.hashKeyRangeForLease().startingHashKey(), + parent.hashKeyRangeForLease().startingHashKey().add(parent.hashKeyRangeForLease().endingHashKey()) + .divide(new BigInteger("2")))); + child1.leaseKey("shard-" + (++leaseCounter)); + Lease child2 = new Lease(); + child2.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child2.hashKeyRange(new HashKeyRangeForLease( + parent.hashKeyRangeForLease().startingHashKey().add(parent.hashKeyRangeForLease().endingHashKey()) + .divide(new BigInteger("2")).add(new BigInteger("1")), + parent.hashKeyRangeForLease().endingHashKey())); + child2.leaseKey("shard-" + (++leaseCounter)); + + child1.parentShardIds(Sets.newHashSet(parent.leaseKey())); + child2.parentShardIds(Sets.newHashSet(parent.leaseKey())); + parent.childShardIds(Lists.newArrayList(child1.leaseKey(), child2.leaseKey())); + +// System.out.println("Parent " + parent + " splits into " + child1 + " and " + child2); + + initialLeases.add(child1); + initialLeases.add(child2); + } + return leaseCounter; + } + + private boolean isHeads() { + return Math.random() <= 0.5; + } + + private boolean isOneFromDiceRoll() { + return Math.random() <= 0.16; + } + + + private enum ReshardType { + SPLIT, + MERGE, + ANY + } + + + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 8be1bb8f..a066ece0 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -22,8 +22,10 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; @@ -32,14 +34,27 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.internal.verification.VerificationModeFactory.atMost; +import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.*; +import java.time.Duration; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.RejectedExecutionException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import com.google.common.base.Joiner; +import com.google.common.collect.Sets; import io.reactivex.plugins.RxJavaPlugins; +import lombok.RequiredArgsConstructor; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,7 +67,14 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.KinesisClientLibException; import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException; +import software.amazon.kinesis.leases.LeaseCleanupManager; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseManagementFactory; @@ -61,8 +83,12 @@ import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; +import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; @@ -71,6 +97,7 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.Checkpointer; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShardRecordProcessor; @@ -89,6 +116,9 @@ public class SchedulerTest { private final String applicationName = "applicationName"; private final String streamName = "streamName"; private final String namespace = "testNamespace"; + private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 5 * 1000L; + private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; + private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; private Scheduler scheduler; private ShardRecordProcessorFactory shardRecordProcessorFactory; @@ -122,24 +152,54 @@ public class SchedulerTest { private Checkpointer checkpoint; @Mock private WorkerStateChangeListener workerStateChangeListener; + @Mock + private MultiStreamTracker multiStreamTracker; + @Mock + private LeaseCleanupManager leaseCleanupManager; + + private Map shardSyncTaskManagerMap; + private Map shardDetectorMap; @Before public void setup() { + shardSyncTaskManagerMap = new HashMap<>(); + shardDetectorMap = new HashMap<>(); shardRecordProcessorFactory = new TestShardRecordProcessorFactory(); checkpointConfig = new CheckpointConfig().checkpointFactory(new TestKinesisCheckpointFactory()); coordinatorConfig = new CoordinatorConfig(applicationName).parentShardPollIntervalMillis(100L).workerStateChangeListener(workerStateChangeListener); leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, - workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory()); + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, false)); lifecycleConfig = new LifecycleConfig(); metricsConfig = new MetricsConfig(cloudWatchClient, namespace); processorConfig = new ProcessorConfig(shardRecordProcessorFactory); retrievalConfig = new RetrievalConfig(kinesisClient, streamName, applicationName) .retrievalFactory(retrievalFactory); + final List streamConfigList = new ArrayList() {{ + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream2:2"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream2:3"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + }}; + + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList); + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()) + .thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); - when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(MetricsFactory.class))).thenReturn(recordsPublisher); + when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(new HierarchicalShardSyncer()); + when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); + when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(StreamConfig.class), any(MetricsFactory.class))).thenReturn(recordsPublisher); + when(shardDetector.streamIdentifier()).thenReturn(mock(StreamIdentifier.class)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); @@ -162,9 +222,9 @@ public class SchedulerTest { final String shardId = "shardId-000000000000"; final String concurrencyToken = "concurrencyToken"; final ShardInfo shardInfo = new ShardInfo(shardId, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory); + final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager); assertNotNull(shardConsumer1); - final ShardConsumer shardConsumer2 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory); + final ShardConsumer shardConsumer2 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager); assertNotNull(shardConsumer2); assertSame(shardConsumer1, shardConsumer2); @@ -172,7 +232,7 @@ public class SchedulerTest { final String anotherConcurrencyToken = "anotherConcurrencyToken"; final ShardInfo shardInfo2 = new ShardInfo(shardId, anotherConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - final ShardConsumer shardConsumer3 = scheduler.createOrGetShardConsumer(shardInfo2, shardRecordProcessorFactory); + final ShardConsumer shardConsumer3 = scheduler.createOrGetShardConsumer(shardInfo2, shardRecordProcessorFactory, leaseCleanupManager); assertNotNull(shardConsumer3); assertNotSame(shardConsumer1, shardConsumer3); @@ -194,7 +254,7 @@ public class SchedulerTest { final List secondShardInfo = Collections.singletonList( new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber)); - final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null); + final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null, null); when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo); when(checkpoint.getCheckpointObject(eq(shardId))).thenReturn(firstCheckpoint); @@ -204,9 +264,9 @@ public class SchedulerTest { schedulerSpy.runProcessLoop(); schedulerSpy.runProcessLoop(); - verify(schedulerSpy).buildConsumer(same(initialShardInfo.get(0)), eq(shardRecordProcessorFactory)); - verify(schedulerSpy, never()).buildConsumer(same(firstShardInfo.get(0)), eq(shardRecordProcessorFactory)); - verify(schedulerSpy, never()).buildConsumer(same(secondShardInfo.get(0)), eq(shardRecordProcessorFactory)); + verify(schedulerSpy).buildConsumer(same(initialShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)); + verify(schedulerSpy, never()).buildConsumer(same(firstShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)); + verify(schedulerSpy, never()).buildConsumer(same(secondShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)); verify(checkpoint).getCheckpointObject(eq(shardId)); } @@ -222,10 +282,10 @@ public class SchedulerTest { ExtendedSequenceNumber.TRIM_HORIZON); final ShardInfo shardInfo1 = new ShardInfo(shard1, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - final ShardConsumer shardConsumer0 = scheduler.createOrGetShardConsumer(shardInfo0, shardRecordProcessorFactory); + final ShardConsumer shardConsumer0 = scheduler.createOrGetShardConsumer(shardInfo0, shardRecordProcessorFactory, leaseCleanupManager); final ShardConsumer shardConsumer0WithAnotherConcurrencyToken = - scheduler.createOrGetShardConsumer(shardInfo0WithAnotherConcurrencyToken, shardRecordProcessorFactory); - final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo1, shardRecordProcessorFactory); + scheduler.createOrGetShardConsumer(shardInfo0WithAnotherConcurrencyToken, shardRecordProcessorFactory, leaseCleanupManager); + final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo1, shardRecordProcessorFactory, leaseCleanupManager); Set shards = new HashSet<>(); shards.add(shardInfo0); @@ -242,27 +302,504 @@ public class SchedulerTest { @Test public final void testInitializationFailureWithRetries() throws Exception { doNothing().when(leaseCoordinator).initialize(); - when(shardDetector.listShards()).thenThrow(new RuntimeException()); - + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenThrow(new RuntimeException()); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true)); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); scheduler.run(); - verify(shardDetector, times(coordinatorConfig.maxInitializationAttempts())).listShards(); + verify(dynamoDBLeaseRefresher, times(coordinatorConfig.maxInitializationAttempts())).isLeaseTableEmpty(); } @Test public final void testInitializationFailureWithRetriesWithConfiguredMaxInitializationAttempts() throws Exception { final int maxInitializationAttempts = 5; coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); doNothing().when(leaseCoordinator).initialize(); - when(shardDetector.listShards()).thenThrow(new RuntimeException()); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenThrow(new RuntimeException()); scheduler.run(); // verify initialization was retried for maxInitializationAttempts times - verify(shardDetector, times(maxInitializationAttempts)).listShards(); + verify(dynamoDBLeaseRefresher, times(maxInitializationAttempts)).isLeaseTableEmpty(); + } + + @Test + public final void testMultiStreamInitialization() throws ProvisionedThroughputException, DependencyException { + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, true)); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + scheduler.initialize(); + shardDetectorMap.values().stream() + .forEach(shardDetector -> verify(shardDetector, times(1)).listShards()); + shardSyncTaskManagerMap.values().stream() + .forEach(shardSyncTM -> verify(shardSyncTM, times(1)).hierarchicalShardSyncer()); + } + + @Test + public final void testMultiStreamInitializationWithFailures() { + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, true)); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + scheduler.initialize(); + // Note : As of today we retry for all streams in the next attempt. Hence the retry for each stream will vary. + // At the least we expect 2 retries for each stream. Since there are 4 streams, we expect at most + // the number of calls to be 5. + shardDetectorMap.values().stream() + .forEach(shardDetector -> verify(shardDetector, atLeast(2)).listShards()); + shardDetectorMap.values().stream() + .forEach(shardDetector -> verify(shardDetector, atMost(5)).listShards()); + shardSyncTaskManagerMap.values().stream() + .forEach(shardSyncTM -> verify(shardSyncTM, atLeast(2)).hierarchicalShardSyncer()); + shardSyncTaskManagerMap.values().stream() + .forEach(shardSyncTM -> verify(shardSyncTM, atMost(5)).hierarchicalShardSyncer()); + } + + + @Test + public final void testMultiStreamConsumersAreBuiltOncePerAccountStreamShard() throws KinesisClientLibException { + final String shardId = "shardId-000000000000"; + final String concurrencyToken = "concurrencyToken"; + final ExtendedSequenceNumber firstSequenceNumber = ExtendedSequenceNumber.TRIM_HORIZON; + final ExtendedSequenceNumber secondSequenceNumber = new ExtendedSequenceNumber("1000"); + final ExtendedSequenceNumber finalSequenceNumber = new ExtendedSequenceNumber("2000"); + + final List initialShardInfo = multiStreamTracker.streamConfigList().stream() + .map(sc -> new ShardInfo(shardId, concurrencyToken, null, firstSequenceNumber, + sc.streamIdentifier().serialize())).collect(Collectors.toList()); + final List firstShardInfo = multiStreamTracker.streamConfigList().stream() + .map(sc -> new ShardInfo(shardId, concurrencyToken, null, secondSequenceNumber, + sc.streamIdentifier().serialize())).collect(Collectors.toList()); + final List secondShardInfo = multiStreamTracker.streamConfigList().stream() + .map(sc -> new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber, + sc.streamIdentifier().serialize())).collect(Collectors.toList()); + + final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null, null); + + when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo); + when(checkpoint.getCheckpointObject(anyString())).thenReturn(firstCheckpoint); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + Scheduler schedulerSpy = spy(scheduler); + schedulerSpy.runProcessLoop(); + schedulerSpy.runProcessLoop(); + schedulerSpy.runProcessLoop(); + + initialShardInfo.stream().forEach( + shardInfo -> verify(schedulerSpy).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), same(leaseCleanupManager))); + firstShardInfo.stream().forEach( + shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager))); + secondShardInfo.stream().forEach( + shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager))); + + } + + @Test + public final void testMultiStreamNoStreamsAreSyncedWhenStreamsAreNotRefreshed() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Assert.assertTrue("SyncedStreams should be empty", syncedStreams.isEmpty()); + Assert.assertEquals(new HashSet(streamConfigList1), new HashSet(scheduler.currentStreamConfigMap().values())); + } + + @Test + public final void testMultiStreamOnlyNewStreamsAreSynced() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.range(5, 7).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(streamConfigList2), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyAutoDeletionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamStaleStreamsAreNotDeletedImmediately(true, false); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyNoDeletionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy()); + testMultiStreamStaleStreamsAreNotDeletedImmediately(false, true); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiersForLeaseCleanup() { + return null; + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamStaleStreamsAreNotDeletedImmediately(false, false); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy2() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiersForLeaseCleanup() { + return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(ArrayList::new)); + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamStaleStreamsAreNotDeletedImmediately(true, false); + } + + private final void testMultiStreamStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion, + boolean onlyStreamsDeletionNotLeases) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedPendingStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(HashSet::new)); + Set expectedSyncedStreams = onlyStreamsDeletionNotLeases ? expectedPendingStreams : Sets.newHashSet(); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(onlyStreamsDeletionNotLeases ? streamConfigList2 : streamConfigList1), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams : Sets.newHashSet(), + scheduler.staleStreamDeletionMap().keySet()); + } + + @Test + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithAutoDetectionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null); + } + + @Test + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiersForLeaseCleanup() { + return null; + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + HashSet currentStreamConfigMapOverride = IntStream.range(1, 5).mapToObj( + streamId -> new StreamConfig(StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(HashSet::new)); + testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(false, currentStreamConfigMapOverride); + } + + @Test + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy2() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiersForLeaseCleanup() { + return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(ArrayList::new)); + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null); + } + + private final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(boolean expectSyncedStreams, Set currentStreamConfigMapOverride) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectSyncedStreams ? expectedSyncedStreams : Sets.newHashSet(), syncedStreams); + Assert.assertEquals(currentStreamConfigMapOverride == null ? Sets.newHashSet(streamConfigList2) : currentStreamConfigMapOverride, + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(Sets.newHashSet(), + scheduler.staleStreamDeletionMap().keySet()); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithAutoDetectionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithNoDeletionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy()); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false, true); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiersForLeaseCleanup() { + return null; + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false, false); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy2() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiersForLeaseCleanup() { + return IntStream.range(1, 3) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(ArrayList::new)); + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false); + } + + private final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion, + boolean onlyStreamsNoLeasesDeletion) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams; + Set expectedPendingStreams = IntStream.range(1, 3) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + + if(onlyStreamsNoLeasesDeletion) { + expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + } else { + expectedSyncedStreams = IntStream.range(5, 7) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + } + + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + List expectedCurrentStreamConfigs; + if(onlyStreamsNoLeasesDeletion) { + expectedCurrentStreamConfigs = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + } else { + expectedCurrentStreamConfigs = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + } + Assert.assertEquals(Sets.newHashSet(expectedCurrentStreamConfigs), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams: Sets.newHashSet(), + scheduler.staleStreamDeletionMap().keySet()); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreDeletedAfterDefermentPeriod() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(streamConfigList2), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(Sets.newHashSet(), + scheduler.staleStreamDeletionMap().keySet()); + } + + @Test + public final void testInitializationWaitsWhenLeaseTableIsEmpty() throws Exception { + final int maxInitializationAttempts = 1; + coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts); + coordinatorConfig.skipShardSyncAtWorkerInitializationIfLeasesExist(false); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + + doNothing().when(leaseCoordinator).initialize(); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + + long startTime = System.currentTimeMillis(); + scheduler.shouldInitiateLeaseSync(); + long endTime = System.currentTimeMillis(); + + assertTrue(endTime - startTime > MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + assertTrue(endTime - startTime < (MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS + LEASE_TABLE_CHECK_FREQUENCY_MILLIS)); + } + + @Test + public final void testInitializationDoesntWaitWhenLeaseTableIsNotEmpty() throws Exception { + final int maxInitializationAttempts = 1; + coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts); + coordinatorConfig.skipShardSyncAtWorkerInitializationIfLeasesExist(false); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + + doNothing().when(leaseCoordinator).initialize(); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); + + long startTime = System.currentTimeMillis(); + scheduler.shouldInitiateLeaseSync(); + long endTime = System.currentTimeMillis(); + + assertTrue(endTime - startTime < MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); } @Test @@ -498,9 +1035,20 @@ public class SchedulerTest { } }; } + + @Override + public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { + return shardRecordProcessor(); + } + } + @RequiredArgsConstructor private class TestKinesisLeaseManagementFactory implements LeaseManagementFactory { + + private final boolean shardSyncFirstAttemptFailure; + private final boolean shouldReturnDefaultShardSyncTaskmanager; + @Override public LeaseCoordinator createLeaseCoordinator(MetricsFactory metricsFactory) { return leaseCoordinator; @@ -511,6 +1059,29 @@ public class SchedulerTest { return shardSyncTaskManager; } + @Override + public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, + StreamConfig streamConfig) { + if(shouldReturnDefaultShardSyncTaskmanager) { + return shardSyncTaskManager; + } + final ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); + final ShardDetector shardDetector = mock(ShardDetector.class); + shardSyncTaskManagerMap.put(streamConfig.streamIdentifier(), shardSyncTaskManager); + shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector); + when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + final HierarchicalShardSyncer hierarchicalShardSyncer = new HierarchicalShardSyncer(); + when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(hierarchicalShardSyncer); + when(shardDetector.streamIdentifier()).thenReturn(streamConfig.streamIdentifier()); + when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); + if(shardSyncFirstAttemptFailure) { + when(shardDetector.listShards()) + .thenThrow(new RuntimeException("Service Exception")) + .thenReturn(Collections.EMPTY_LIST); + } + return shardSyncTaskManager; + } + @Override public DynamoDBLeaseRefresher createLeaseRefresher() { return dynamoDBLeaseRefresher; @@ -520,6 +1091,16 @@ public class SchedulerTest { public ShardDetector createShardDetector() { return shardDetector; } + + @Override + public ShardDetector createShardDetector(StreamConfig streamConfig) { + return shardDetectorMap.get(streamConfig.streamIdentifier()); + } + + @Override + public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) { + return leaseCleanupManager; + } } private class TestKinesisCheckpointFactory implements CheckpointFactory { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java index 233ce724..81a49839 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java @@ -19,6 +19,7 @@ import java.util.List; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -54,6 +55,7 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { DELETELEASE(9), DELETEALL(10), UPDATELEASE(11), + LISTLEASESFORSTREAM(12), NONE(Integer.MIN_VALUE); private Integer index; @@ -129,6 +131,13 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { return leaseRefresher.waitUntilLeaseTableExists(secondsBetweenPolls, timeoutSeconds); } + @Override + public List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throwExceptions("listLeasesForStream", ExceptionThrowingLeaseRefresherMethods.LISTLEASESFORSTREAM); + + return leaseRefresher.listLeasesForStream(streamIdentifier); + } + @Override public List listLeases() throws DependencyException, InvalidStateException, ProvisionedThroughputException { @@ -186,11 +195,11 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { } @Override - public Lease getLease(String shardId) + public Lease getLease(String leaseKey) throws DependencyException, InvalidStateException, ProvisionedThroughputException { throwExceptions("getLease", ExceptionThrowingLeaseRefresherMethods.GETLEASE); - return leaseRefresher.getLease(shardId); + return leaseRefresher.getLease(leaseKey); } @Override @@ -207,7 +216,7 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { } @Override - public ExtendedSequenceNumber getCheckpoint(final String shardId) + public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException { return null; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index 23d2e423..c390987c 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -21,6 +21,7 @@ package software.amazon.kinesis.leases; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; import static org.mockito.Matchers.any; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; @@ -32,6 +33,7 @@ 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; @@ -39,10 +41,12 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.function.Consumer; import java.util.stream.Collectors; -import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.Validate; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -53,8 +57,12 @@ import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.model.HashKeyRange; import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.awssdk.services.kinesis.model.ShardFilterType; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.leases.exceptions.DependencyException; @@ -62,6 +70,8 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.NullMetricsScope; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import static software.amazon.kinesis.leases.HierarchicalShardSyncer.MemoizationContext; + @RunWith(MockitoJUnitRunner.class) // CHECKSTYLE:IGNORE JavaNCSS FOR NEXT 800 LINES public class HierarchicalShardSyncerTest { @@ -74,8 +84,11 @@ public class HierarchicalShardSyncerTest { private static final int EXPONENT = 128; private static final String LEASE_OWNER = "TestOwnere"; private static final MetricsScope SCOPE = new NullMetricsScope(); + private static final boolean MULTISTREAM_MODE_ON = true; + private static final String STREAM_IDENTIFIER = "acc:stream:1"; + private static final HierarchicalShardSyncer.MultiStreamArgs MULTI_STREAM_ARGS = new HierarchicalShardSyncer.MultiStreamArgs( + MULTISTREAM_MODE_ON, StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); - private final boolean cleanupLeasesOfCompletedShards = true; private final boolean ignoreUnexpectedChildShards = false; private HierarchicalShardSyncer hierarchicalShardSyncer; @@ -93,6 +106,12 @@ public class HierarchicalShardSyncerTest { @Before public void setup() { hierarchicalShardSyncer = new HierarchicalShardSyncer(); + when(shardDetector.streamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("stream")); + } + + private void setupMultiStream() { + hierarchicalShardSyncer = new HierarchicalShardSyncer(true, STREAM_IDENTIFIER); + when(shardDetector.streamIdentifier()).thenReturn(StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); } /** @@ -102,9 +121,22 @@ public class HierarchicalShardSyncerTest { public void testDetermineNewLeasesToCreateNoShards() { final List shards = Collections.emptyList(); final List leases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + assertThat(HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, + INITIAL_POSITION_LATEST).isEmpty(), equalTo(true)); + } - assertThat(HierarchicalShardSyncer.determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST).isEmpty(), - equalTo(true)); + /** + * Test determineNewLeasesToCreate() where there are no shards for MultiStream + */ + @Test public void testDetermineNewLeasesToCreateNoShardsForMultiStream() { + final List shards = Collections.emptyList(); + final List leases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + + assertThat(HierarchicalShardSyncer + .determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, INITIAL_POSITION_LATEST, + new HashSet<>(), MULTI_STREAM_ARGS).isEmpty(), equalTo(true)); } /** @@ -119,9 +151,12 @@ public class HierarchicalShardSyncerTest { final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); final List currentLeases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_LATEST); + validateHashRangeinLease(newLeases); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); @@ -129,32 +164,111 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); } + /** + * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed + */ + @Test + public void testDetermineNewLeasesToCreate0Leases0ReshardsForMultiStream() { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); + final List currentLeases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS); + validateHashRangeinLease(newLeases); + final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set expectedLeaseIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); + + assertThat(newLeases.size(), equalTo(expectedLeaseIds.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseIds)); + } + + /** + * Test determineNewLeasesToCreate() where there is one lease and no resharding operations have been performed, but + * one of the shards was marked as inconsistent. + */ + @Test + public void testDetermineNewLeasesToCreate0Leases0Reshards1Inconsistent() { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + final String shardId2 = "shardId-2"; + final String shardId3 = "shardId-3"; + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId3, null, null, sequenceRange)); + final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); + + final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); + final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); + final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); + + Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, + INITIAL_POSITION_LATEST, inconsistentShardIds); + validateHashRangeinLease(newLeases); + final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); + assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + } + /** * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but * one of the shards was marked as inconsistent. */ @Test - public void testDetermineNewLeasesToCreate0Leases0Reshards1Inconsistent() { + public void testDetermineNewLeasesToCreate0Leases0Reshards1InconsistentMultiStream() { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; final String shardId2 = "shardId-2"; + final String shardId3 = "shardId-3"; final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId3, null, null, sequenceRange)); + final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); - final List currentLeases = Collections.emptyList(); + final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); + final List currentLeases = new ArrayList(createMultiStreamLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo")); final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST, inconsistentShardIds); + Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, + INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); + validateHashRangeinLease(newLeases); + final Set expectedLeaseShardIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); } + private void validateHashRangeinLease(List leases) { + final Consumer leaseValidation = lease -> { + Validate.notNull(lease.hashKeyRangeForLease()); + Validate.isTrue(lease.hashKeyRangeForLease().startingHashKey() + .compareTo(lease.hashKeyRangeForLease().endingHashKey()) < 0); + }; + leases.forEach(lease -> leaseValidation.accept(lease)); + } + /** * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) */ @@ -186,7 +300,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -196,6 +310,7 @@ public class HierarchicalShardSyncerTest { final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); + validateHashRangeinLease(requestLeases); assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); assertThat(requestLeaseKeys, equalTo(expectedShardIds)); assertThat(extendedSequenceNumbers.size(), equalTo(1)); @@ -208,6 +323,46 @@ public class HierarchicalShardSyncerTest { } + @Test + public void testCheckAndCreateLeasesForShardsIfMissingAtLatestMultiStream() throws Exception { + final List shards = constructShardListForGraphA(); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + + when(shardDetector.listShards()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + setupMultiStream(); + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + final Set expectedShardIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); + + final List requestLeases = leaseCaptor.getAllValues(); + final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) + .collect(Collectors.toSet()); + + validateHashRangeinLease(requestLeases); + assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); + assertThat(requestLeaseKeys, equalTo(expectedShardIds)); + assertThat(extendedSequenceNumbers.size(), equalTo(1)); + + extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); + + verify(shardDetector).listShards(); + verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + + } + + private List toMultiStreamLeaseList(List shardIdBasedLeases) { + return shardIdBasedLeases.stream().map(s -> STREAM_IDENTIFIER + ":" + s) + .collect(Collectors.toList()); + } + /** * Test checkAndCreateLeaseForNewShards with a pre-fetched list of shards. In this scenario, shardDetector.listShards() * should never be called. @@ -223,7 +378,8 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); + latestShards, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -237,6 +393,45 @@ public class HierarchicalShardSyncerTest { assertThat(requestLeaseKeys, equalTo(expectedShardIds)); assertThat(extendedSequenceNumbers.size(), equalTo(1)); + validateHashRangeinLease(requestLeases); + + extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); + + verify(shardDetector, never()).listShards(); + verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + } + + /** + * Test checkAndCreateLeaseForNewShards with a pre-fetched list of shards. In this scenario, shardDetector.listShards() + * should never be called. + */ + @Test + public void testCheckAndCreateLeasesForShardsWithShardListMultiStream() throws Exception { + final List latestShards = constructShardListForGraphA(); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + when(shardDetector.listShards()).thenReturn(latestShards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + setupMultiStream(); + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + latestShards, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + final Set expectedShardIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); + + final List requestLeases = leaseCaptor.getAllValues(); + final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) + .collect(Collectors.toSet()); + + assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); + assertThat(requestLeaseKeys, equalTo(expectedShardIds)); + assertThat(extendedSequenceNumbers.size(), equalTo(1)); + validateHashRangeinLease(requestLeases); extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); verify(shardDetector, never()).listShards(); @@ -259,7 +454,8 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE, new ArrayList()); + new ArrayList(), false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>(); @@ -267,7 +463,7 @@ public class HierarchicalShardSyncerTest { final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - + validateHashRangeinLease(requestLeases); assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); assertThat(extendedSequenceNumbers.size(), equalTo(0)); @@ -276,14 +472,180 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } + /* + * 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 testCheckAndCreateLeasesForNewShardsAtTrimHorizon() throws Exception { - testCheckAndCreateLeaseForShardsIfMissing(constructShardListForGraphA(), INITIAL_POSITION_TRIM_HORIZON); + 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 testCheckAndCreateLeasesForNewShardsAtTimestamp() throws Exception { - testCheckAndCreateLeaseForShardsIfMissing(constructShardListForGraphA(), INITIAL_POSITION_AT_TIMESTAMP); + 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 testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() 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); + } + + /* + * 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.shardId())).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.shardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, 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(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.shardId())).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 testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() 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.shardId())).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); } @Test(expected = KinesisClientLibIOException.class) @@ -299,7 +661,28 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, false, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + } finally { + verify(shardDetector).listShards(); + verify(dynamoDBLeaseRefresher, never()).listLeases(); + } + } + + @Test(expected = KinesisClientLibIOException.class) + public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenForMultiStream() throws Exception { + final List shards = new ArrayList<>(constructShardListForGraphA()); + final SequenceNumberRange range = shards.get(0).sequenceNumberRange().toBuilder().endingSequenceNumber(null) + .build(); + final Shard shard = shards.get(3).toBuilder().sequenceNumberRange(range).build(); + shards.remove(3); + shards.add(3, shard); + + when(shardDetector.listShards()).thenReturn(shards); + setupMultiStream(); + try { + hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, + INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -334,7 +717,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, true, SCOPE); + SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -354,6 +737,51 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } + @Test + public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildrenMultiStream() throws Exception { + final List shards = new ArrayList<>(constructShardListForGraphA()); + final Shard shard = shards.get(5); + assertThat(shard.shardId(), equalTo("shardId-5")); + + shards.remove(5); + + // shardId-5 in graph A has two children (shardId-9 and shardId-10). if shardId-5 + // is not closed, those children should be ignored when syncing shards, no leases + // should be obtained for them, and we should obtain a lease on the still-open + // parent. + shards.add(5, + shard.toBuilder() + .sequenceNumberRange(shard.sequenceNumberRange().toBuilder().endingSequenceNumber(null).build()) + .build()); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + + when(shardDetector.listShards()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + setupMultiStream(); + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + final List leases = leaseCaptor.getAllValues(); + final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set leaseSequenceNumbers = leases.stream().map(Lease::checkpoint) + .collect(Collectors.toSet()); + + final Set expectedShardIds = new HashSet<>(toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-5", "shardId-8"))); + + assertThat(leaseKeys.size(), equalTo(expectedShardIds.size())); + assertThat(leaseKeys, equalTo(expectedShardIds)); + assertThat(leaseSequenceNumbers.size(), equalTo(1)); + + leaseSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); + + verify(shardDetector).listShards(); + verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + } + @Test public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShard() throws Exception { testCheckAndCreateLeasesForNewShardsAndClosedShard(ExtendedSequenceNumber.TRIM_HORIZON, @@ -390,11 +818,11 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); @@ -403,125 +831,19 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); - // Second call: Leases present, with shardId-0 being at ShardEnd causing cleanup. + // Second call: Leases present, no leases should be deleted. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - final Set expectedShardIds = new HashSet<>(Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); + assertThat(deleteLeases.size(), equalTo(0)); verify(shardDetector, times(2)).listShards(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(2)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); - } - - @Test(expected = DependencyException.class) - public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithDeleteLeaseExceptions() - throws Exception { - testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions(ExtendedSequenceNumber.TRIM_HORIZON, - INITIAL_POSITION_TRIM_HORIZON); - } - - @Test(expected = DependencyException.class) - public void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithDeleteLeaseExceptions() - throws Exception { - testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions(ExtendedSequenceNumber.AT_TIMESTAMP, - INITIAL_POSITION_AT_TIMESTAMP); - } - - private void testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions( - final ExtendedSequenceNumber sequenceNumber, final InitialPositionInStreamExtended position) - throws Exception { - final String shardIdPrefix = "shardId-%d"; - final List shards = constructShardListForGraphA(); - final List leases = createLeasesFromShards(shards, sequenceNumber, LEASE_OWNER); - - // Marking shardId-0 as ShardEnd. - leases.stream().filter(lease -> String.format(shardIdPrefix, 0).equals(lease.leaseKey())).findFirst() - .ifPresent(lease -> lease.checkpoint(ExtendedSequenceNumber.SHARD_END)); - - // Marking child of shardId-0 to be processed and not at TRIM_HORIZON. - leases.stream().filter(lease -> String.format(shardIdPrefix, 6).equals(lease.leaseKey())).findFirst() - .ifPresent(lease -> lease.checkpoint(new ExtendedSequenceNumber("1"))); - - final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()).thenReturn(leases); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())).thenReturn(true); - doThrow(new DependencyException(new Throwable("Throw for DeleteLease"))).doNothing() - .when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); - - // Initial call: Call to create leases. - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); - - final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); - - assertThat(createLeases, equalTo(expectedCreateLeases)); - - verify(shardDetector, times(1)).listShards(); - verify(dynamoDBLeaseRefresher, times(1)).listLeases(); - verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); - - try { - // Second call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete fails. - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); - } finally { - List deleteLeases = leaseDeleteCaptor.getAllValues(); - Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - final Set expectedShardIds = new HashSet<>( - Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); - - verify(shardDetector, times(2)).listShards(); - verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, times(2)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); - - // Final call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete passes. - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); - - deleteLeases = leaseDeleteCaptor.getAllValues(); - - shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint).collect(Collectors.toSet()); - - assertThat(deleteLeases.size(), equalTo(2)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); - - verify(shardDetector, times(3)).listShards(); - verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, times(3)).listLeases(); - verify(dynamoDBLeaseRefresher, times(2)).deleteLease(any(Lease.class)); - } } @Test(expected = DependencyException.class) @@ -554,20 +876,18 @@ public class HierarchicalShardSyncerTest { .ifPresent(lease -> lease.checkpoint(new ExtendedSequenceNumber("1"))); final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()) .thenThrow(new DependencyException(new Throwable("Throw for ListLeases"))) .thenReturn(Collections.emptyList()).thenReturn(leases); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())).thenReturn(true); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); try { // Initial call: Call to create leases. Fails on ListLeases hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -577,10 +897,10 @@ public class HierarchicalShardSyncerTest { // Second call: Leases not present, leases will be created. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); @@ -592,26 +912,17 @@ public class HierarchicalShardSyncerTest { // Final call: Leases present, belongs to TestOwner, shardId-0 is at ShardEnd should be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); - - final List deleteLeases = leaseDeleteCaptor.getAllValues(); - final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Collections.singletonList(String.format(shardIdPrefix, 0))); final Set expectedSequenceNumbers = new HashSet<>( Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); - verify(shardDetector, times(3)).listShards(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } } @@ -645,20 +956,18 @@ public class HierarchicalShardSyncerTest { .ifPresent(lease -> lease.checkpoint(new ExtendedSequenceNumber("1"))); final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()) .thenReturn(Collections.emptyList()).thenReturn(leases); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())) .thenThrow(new DependencyException(new Throwable("Throw for CreateLease"))).thenReturn(true); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); try { // Initial call: No leases present, create leases. Create lease Fails hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -667,10 +976,11 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); verify(shardDetector, times(2)).listShards(); @@ -682,27 +992,13 @@ public class HierarchicalShardSyncerTest { // Final call: Leases are present, shardId-0 is at ShardEnd needs to be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); - - final List deleteLeases = leaseDeleteCaptor.getAllValues(); - final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - final Set expectedShardIds = new HashSet<>( - Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, times(3)).listShards(); verify(dynamoDBLeaseRefresher, times(1 + expectedCreateLeases.size())) .createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } } @@ -711,6 +1007,11 @@ public class HierarchicalShardSyncerTest { return createLeasesFromShards(Collections.singletonList(shard), checkpoint, leaseOwner).get(0); } + private MultiStreamLease createMultiStreamLeaseFromShard(final Shard shard, final ExtendedSequenceNumber checkpoint, + final String leaseOwner) { + return createMultiStreamLeasesFromShards(Collections.singletonList(shard), checkpoint, leaseOwner).get(0); + } + private List createLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, final String leaseOwner) { return shards.stream().map(shard -> { @@ -722,76 +1023,83 @@ public class HierarchicalShardSyncerTest { parentShardIds.add(shard.adjacentParentShardId()); } return new Lease(shard.shardId(), leaseOwner, 0L, UUID.randomUUID(), 0L, checkpoint, null, 0L, - parentShardIds); + parentShardIds, new HashSet<>(), null, HashKeyRangeForLease.fromHashKeyRange(shard.hashKeyRange())); }).collect(Collectors.toList()); } - @Test - public void testCleanUpGarbageLeaseForNonExistentShard() throws Exception { - final List shards = constructShardListForGraphA(); - final String garbageShardId = "shardId-garbage-001"; - final Shard garbageShard = ShardObjectHelper.newShard(garbageShardId, null, null, - ShardObjectHelper.newSequenceNumberRange("101", null)); - final Lease garbageLease = createLeaseFromShard(garbageShard, new ExtendedSequenceNumber("99"), LEASE_OWNER); - final List leases = new ArrayList<>( - createLeasesFromShards(shards, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER)); - leases.add(garbageLease); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(leases); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); - - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); - - assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); - assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); - - verify(shardDetector, times(2)).listShards(); - verify(dynamoDBLeaseRefresher).listLeases(); - verify(dynamoDBLeaseRefresher).deleteLease(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); + private List createMultiStreamLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, + final String leaseOwner) { + return shards.stream().map(shard -> { + final Set parentShardIds = new HashSet<>(); + if (StringUtils.isNotEmpty(shard.parentShardId())) { + parentShardIds.add(shard.parentShardId()); + } + if (StringUtils.isNotEmpty(shard.adjacentParentShardId())) { + parentShardIds.add(shard.adjacentParentShardId()); + } + final MultiStreamLease msLease = new MultiStreamLease(); + msLease.shardId(shard.shardId()); + msLease.leaseOwner(leaseOwner); + msLease.leaseCounter(0L); + msLease.concurrencyToken(UUID.randomUUID()); + msLease.lastCounterIncrementNanos(0L); + msLease.checkpoint(checkpoint); + msLease.parentShardIds(parentShardIds); + msLease.streamIdentifier(STREAM_IDENTIFIER); + return msLease; + }).collect(Collectors.toList()); } private void testCheckAndCreateLeasesForShardsIfMissing(InitialPositionInStreamExtended initialPosition) throws Exception { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; - final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), - ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); + final HashKeyRange range1 = ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, BigInteger.ONE.toString()); + final HashKeyRange range2 = ShardObjectHelper.newHashKeyRange(new BigInteger("2").toString(), ShardObjectHelper.MAX_HASH_KEY); + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("11", null); + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange, range1), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange, range2)); + final Set expectedLeaseKeys = new HashSet<>(Arrays.asList(shardId0, shardId1)); - testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeys); } private void testCheckAndCreateLeaseForShardsIfMissing(final List shards, - final InitialPositionInStreamExtended initialPosition) throws Exception { + final InitialPositionInStreamExtended initialPosition, + final Set expectedLeaseKeys) throws Exception { + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeys, Collections.emptyList()); + } + + private void testCheckAndCreateLeaseForShardsIfMissing(final List shards, + final InitialPositionInStreamExtended initialPosition, + final Set expectedLeaseKeys, + final List existingLeases) throws Exception { + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(shardDetector.listShardsWithFilter(any())).thenReturn(getFilteredShards(shards, initialPosition)); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(existingLeases); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(existingLeases.isEmpty()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - cleanupLeasesOfCompletedShards, false, SCOPE); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set leaseSequenceNumbers = leases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - final Set expectedLeaseKeys = shards.stream().map(Shard::shardId).collect(Collectors.toSet()); + final Set expectedSequenceNumbers = new HashSet<>(Collections .singletonList(new ExtendedSequenceNumber(initialPosition.getInitialPositionInStream().name()))); - assertThat(leases.size(), equalTo(shards.size())); + assertThat(leases.size(), equalTo(expectedLeaseKeys.size())); assertThat(leaseKeys, equalTo(expectedLeaseKeys)); assertThat(leaseSequenceNumbers, equalTo(expectedSequenceNumbers)); - verify(shardDetector).listShards(); - verify(dynamoDBLeaseRefresher, times(shards.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, times(expectedLeaseKeys.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -800,6 +1108,7 @@ public class HierarchicalShardSyncerTest { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; final List currentLeases = new ArrayList<>(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), @@ -811,7 +1120,7 @@ public class HierarchicalShardSyncerTest { final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); for (InitialPositionInStreamExtended initialPosition : initialPositions) { - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, initialPosition); assertThat(newLeases.size(), equalTo(2)); @@ -823,285 +1132,877 @@ public class HierarchicalShardSyncerTest { } } + /** + * Tests that leases are not created for closed shards. + */ @Test public void testDetermineNewLeasesToCreateIgnoreClosedShard() { final String lastShardId = "shardId-1"; - final List currentLeases = new ArrayList<>(); - final List shards = Arrays.asList( + final List shardsWithoutLeases = Arrays.asList( ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("303", "404")), ShardObjectHelper.newShard(lastShardId, null, null, ShardObjectHelper.newSequenceNumberRange("405", null))); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard("shardId-2", null, + null, ShardObjectHelper.newSequenceNumberRange("202", "302"))); + + final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); + final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); + final Set inconsistentShardIds = Collections.emptySet(); + + Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); assertThat(newLeases.size(), equalTo(1)); assertThat(newLeases.get(0).leaseKey(), equalTo(lastShardId)); } -// /** -// * 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) -// * Current leases: (3, 4, 5) -// */ + /** + * 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: LATEST + * Expected leases: (2, 6) + */ @Test - public void testDetermineNewLeasesToCreateSplitMergeLatest1() { + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange1() { final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), - newLease("shardId-5")); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); - + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); 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); - - assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); - for (Lease lease : newLeases) { - assertThat("Unexpected lease: " + lease, expectedShardIdCheckpointMap.containsKey(lease.leaseKey()), - equalTo(true)); - assertThat(lease.checkpoint(), equalTo(expectedShardIdCheckpointMap.get(lease.leaseKey()))); - } - } - -// /** -// * 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) -// * Current leases: (4, 5, 7) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeLatest2() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), - newLease("shardId-7")); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); - - 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); - - assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); - for (Lease lease : newLeases) { - assertThat("Unexpected lease: " + lease, expectedShardIdCheckpointMap.containsKey(lease.leaseKey()), - equalTo(true)); - assertThat(lease.checkpoint(), equalTo(expectedShardIdCheckpointMap.get(lease.leaseKey()))); - } - } - -// /** -// * 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) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeHorizon1() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), - newLease("shardId-5")); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_TRIM_HORIZON); - - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", - "shardId-6", "shardId-7", "shardId-8", "shardId-9", "shardId-10")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.TRIM_HORIZON)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * 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) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeHorizon2() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), - newLease("shardId-7")); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_TRIM_HORIZON); - - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>( - Arrays.asList("shardId-8", "shardId-9", "shardId-10", "shardId-6", "shardId-0", "shardId-1")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.TRIM_HORIZON)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) -// * For shard graph B (see the construct method doc for structure). -// * -// * Current leases: empty set -// */ - @Test - public void testDetermineNewLeasesToCreateGraphBNoInitialLeasesTrim() { - final List shards = constructShardListForGraphB(); - final List currentLeases = new ArrayList<>(); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_TRIM_HORIZON); - - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.TRIM_HORIZON)); - final Set expectedLeaseKeys = IntStream.range(0, 11).mapToObj(id -> String.format("shardId-%d", id)) - .collect(Collectors.toSet()); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * 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: (3, 4, 5) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeAtTimestamp1() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), - newLease("shardId-5")); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_AT_TIMESTAMP); - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", - "shardId-6", "shardId-7", "shardId-8", "shardId-9", "shardId-10")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.AT_TIMESTAMP)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * 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, 7) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeAtTimestamp2() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), - newLease("shardId-7")); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_AT_TIMESTAMP); - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>( - Arrays.asList("shardId-0", "shardId-1", "shardId-6", "shardId-8", "shardId-9", "shardId-10")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.AT_TIMESTAMP)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, 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: (4, 5, 7) + * Initial position: LATEST + * Expected leases: (6) + * */ @Test - public void testDetermineNewLeasesToCreateGraphBNoInitialLeasesAtTimestamp() { + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(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: (2, 6) + * Initial position: LATEST + * Expected leases: (3, 4, 9, 10) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(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: (4, 9, 10) + * Initial position: LATEST + * Expected leases: (8) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * 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 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); + assertExpectedLeasesAreCreated(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: (4, 5, 6, 7) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, 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: LATEST + * Expected leases: empty set + */ + @Test + public 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 expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, 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: 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); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (6) + * Initial position: LATEST + * Expected leases: (7) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_PartialHashRange() { final List shards = constructShardListForGraphB(); - final List currentLeases = new ArrayList<>(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_AT_TIMESTAMP); - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } - final Set expectedLeaseKeys = IntStream.range(0, shards.size()) - .mapToObj(id -> String.format("shardId-%d", id)).collect(Collectors.toSet()); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.AT_TIMESTAMP)); + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: empty set + * Initial position: LATEST + * Expected leases: (9, 10) + */ + @Test + 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); + assertExpectedLeasesAreCreated(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: (3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2) + */ + @Test + 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); + assertExpectedLeasesAreCreated(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, 7) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) + */ + @Test + 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); + assertExpectedLeasesAreCreated(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: (2, 6) + * Initial position: TRIM_HORIZON + * Expected leases: (3, 4, 5) + */ + @Test + 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); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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); +// assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); +// } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(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); + assertExpectedLeasesAreCreated(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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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); +// assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); +// } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * 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); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + private void assertExpectedLeasesAreCreated(List shards, + List shardIdsOfCurrentLeases, + InitialPositionInStreamExtended initialPosition, + Map expectedShardIdCheckpointMap) { + + final List currentLeases = shardIdsOfCurrentLeases.stream() + .map(shardId -> newLease(shardId)).collect(Collectors.toList()); + + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, initialPosition); + + assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); + for (Lease lease : newLeases) { + assertThat("Unexpected lease: " + lease, expectedShardIdCheckpointMap.containsKey(lease.leaseKey()), + equalTo(true)); + assertThat(lease.checkpoint(), equalTo(expectedShardIdCheckpointMap.get(lease.leaseKey()))); + } } /* @@ -1145,14 +2046,65 @@ public class HierarchicalShardSyncerTest { ShardObjectHelper.newHashKeyRange("800", ShardObjectHelper.MAX_HASH_KEY))); } + /** + * 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.sequenceNumberRange().endingSequenceNumber() == null) + .collect(Collectors.toList()); + case TRIM_HORIZON: + String minSeqNum = shards.stream() + .min(Comparator.comparingLong(s -> Long.parseLong(s.sequenceNumberRange().startingSequenceNumber()))) + .map(s -> s.sequenceNumberRange().startingSequenceNumber()) + .orElseThrow(RuntimeException::new); + return shards.stream() + .filter(s -> s.sequenceNumberRange().startingSequenceNumber().equals(minSeqNum)) + .collect(Collectors.toList()); + case AT_TIMESTAMP: + return shards.stream() + .filter(s -> new Date(Long.parseLong(s.sequenceNumberRange().startingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) <= 0) + .filter(s -> s.sequenceNumberRange().endingSequenceNumber() == null || + new Date(Long.parseLong(s.sequenceNumberRange().endingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) > 0) + .collect(Collectors.toList()); + } + throw new RuntimeException("Unsupported initial position " + initialPosition); + } + + /* + * Helper method to get expected shards for Graph A based on initial position in stream. Shard structure (y-axis is + * epochs): 0 1 2 3 4 5- shards till + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - + * shards from epoch 206 (open - no ending sequenceNumber) + */ + private Set getExpectedLeasesForGraphA(List shards, + ExtendedSequenceNumber sequenceNumber, + InitialPositionInStreamExtended initialPosition) { + final List filteredShards; + if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { + // Lease creation for AT_TIMESTAMP should work the same as for TRIM_HORIZON - ignore shard filters + filteredShards = getFilteredShards(shards, INITIAL_POSITION_TRIM_HORIZON); + } else { + filteredShards = getFilteredShards(shards, initialPosition); + } + return new HashSet<>(createLeasesFromShards(filteredShards, sequenceNumber, null)); + } + // /* // * 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 +// * 0 3 6 9 // * \ / \ / \ / -// * 2 5 8 +// * 2 5 8 // * / \ / \ / \ -// * 1 4 7 10 +// * 1 4 7 10 // */ private List constructShardListForGraphB() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("1000", "1049"); @@ -1180,12 +2132,53 @@ public class HierarchicalShardSyncerTest { ShardObjectHelper.newShard("shardId-10", null, "shardId-8", range6, hashRange1)); } + /** + * 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 void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() { - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); assertThat(HierarchicalShardSyncer .checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST, null, null, @@ -1198,7 +2191,7 @@ public class HierarchicalShardSyncerTest { @Test public void testCheckIfDescendantAndAddNewLeasesForAncestorsTrimmedShard() { final String shardId = "shardId-trimmed"; - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); assertThat(HierarchicalShardSyncer .checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, null, @@ -1213,7 +2206,7 @@ public class HierarchicalShardSyncerTest { final String shardId = "shardId-current"; final Set shardIdsOfCurrentLeases = new HashSet<>(Collections.singletonList(shardId)); final Map newLeaseMap = Collections.emptyMap(); - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); final Map kinesisShards = new HashMap<>(); kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, null, null, null)); @@ -1233,7 +2226,7 @@ public class HierarchicalShardSyncerTest { final String shardId = "shardId-9-1"; final Set shardIdsOfCurrentLeases = Collections.emptySet(); final Map newLeaseMap = Collections.emptyMap(); - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); final Map kinesisShards = new HashMap<>(); kinesisShards.put(parentShardId, ShardObjectHelper.newShard(parentShardId, null, null, null)); @@ -1246,6 +2239,150 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseMap.isEmpty(), equalTo(true)); } + /** + * Tests that when reading from TIP, we use the AT_LATEST shard filter. + * @throws Exception + */ + @Test + public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtLatest() throws Exception { + ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_LATEST).build(); + testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_LATEST, shardFilter); + } + + /** + * Tests that when reading from TRIM, we use the TRIM_HORIZON shard filter. + * @throws Exception + */ + @Test + public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTrimHorizon() throws Exception { + ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build(); + testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_TRIM_HORIZON, shardFilter); + } + + /** + * Tests that when reading from AT_TIMESTAMP, we use the AT_TIMESTAMP shard filter. + * @throws Exception + */ + @Test + public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTimestamp() throws Exception { + ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_TIMESTAMP).timestamp(new Date(1000L).toInstant()).build(); + testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_AT_TIMESTAMP, shardFilter); + } + + public void testEmptyLeaseTableBootstrapUsesListShardsWithFilter(InitialPositionInStreamExtended initialPosition, ShardFilter shardFilter) throws Exception { + final String shardId0 = "shardId-0"; + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, + ShardObjectHelper.newSequenceNumberRange("1", null), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, ShardObjectHelper.MAX_HASH_KEY))); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(shardFilter)).thenReturn(shards); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + verify(shardDetector, atLeast(1)).listShardsWithFilter(shardFilter); + verify(shardDetector, never()).listShards(); + } + + @Test + public void testNonEmptyLeaseTableUsesListShards() throws Exception { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"))); + final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId1, null, null, ShardObjectHelper.newSequenceNumberRange("3", "4"))); + + final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); + when(shardDetector.listShards()).thenReturn(shardsWithoutLeases); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(currentLeases); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + verify(shardDetector, atLeast(1)).listShards(); + } + + /** + * Tries to boostrap empty lease table. Verifies that if we fail to get a complete hash range of shards after three + * retries, we fast fail and throw an exception. + * @throws Exception + */ + @Test(expected = KinesisClientLibIOException.class) + public void testEmptyLeaseTableThrowsExceptionWhenHashRangeIsStillIncompleteAfterRetries() throws Exception { + final List shardsWithIncompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("0", "1")), + ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("2", "3")) + ); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(any(ShardFilter.class))).thenReturn(shardsWithIncompleteHashRange); + + try { + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, ignoreUnexpectedChildShards, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + } finally { + verify(shardDetector, times(3)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. + } + } + + /** + * Tries to bootstrap an empty lease table. Verifies that after getting an incomplete hash range of shards two times + * and a complete hash range the final time, we create the leases. + * @throws Exception + */ + @Test + public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRangeAfterTwoRetries() throws Exception { + final List shardsWithIncompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "69")), + ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("71", ShardObjectHelper.MAX_HASH_KEY)) + ); + final List shardsWithCompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")), + ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY)) + ); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(any(ShardFilter.class))).thenReturn(shardsWithIncompleteHashRange) + .thenReturn(shardsWithIncompleteHashRange).thenReturn(shardsWithCompleteHashRange); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, ignoreUnexpectedChildShards, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + verify(shardDetector, times(3)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. + verify(dynamoDBLeaseRefresher, times(2)).createLeaseIfNotExists(any(Lease.class)); + } + + /** + * Tries to bootstrap an empty lease table. Verifies that leases are created when we have a complete hash range of shards. + * @throws Exception + */ + @Test + public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRange() throws Exception { + final List shardsWithCompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")), + ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY)) + ); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(any(ShardFilter.class))).thenReturn(shardsWithCompleteHashRange); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, ignoreUnexpectedChildShards, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + verify(shardDetector, times(1)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. + verify(dynamoDBLeaseRefresher, times(2)).createLeaseIfNotExists(any(Lease.class)); + } + // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors - two parents, there is a lease for one parent. // */ @@ -1778,5 +2915,4 @@ public class HierarchicalShardSyncerTest { return lease; } - } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java index 1a37f614..d0870d51 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java @@ -15,17 +15,21 @@ package software.amazon.kinesis.leases; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.isA; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.junit.Assert.assertThat; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.LimitExceededException; +import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; +import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; +import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; +import software.amazon.awssdk.services.kinesis.model.Shard; import java.util.ArrayList; import java.util.Arrays; @@ -36,21 +40,16 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; - -import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.awssdk.services.kinesis.model.LimitExceededException; -import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; -import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; -import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; -import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; -import software.amazon.awssdk.services.kinesis.model.Shard; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.isA; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * @@ -143,19 +142,17 @@ public class KinesisShardDetectorTest { } } - @Test(expected = ResourceNotFoundException.class) - public void testListShardsResourceNotFound() { + @Test + public void testListShardsResourceNotFoundReturnsEmptyResponse() { final CompletableFuture future = CompletableFuture.supplyAsync(() -> { throw ResourceNotFoundException.builder().build(); }); - when(client.listShards(any(ListShardsRequest.class))).thenReturn(future); - try { - shardDetector.listShards(); - } finally { - verify(client).listShards(any(ListShardsRequest.class)); - } + List shards = shardDetector.listShards(); + + Assert.assertEquals(0, shards.size()); + verify(client).listShards(any(ListShardsRequest.class)); } @Test diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java index ee38116f..8f825875 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java @@ -20,6 +20,7 @@ import java.util.UUID; import lombok.Setter; import lombok.experimental.Accessors; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @Setter @@ -34,9 +35,13 @@ public class LeaseBuilder { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); + private byte[] pendingCheckpointState; + private HashKeyRangeForLease hashKeyRangeForLease; public Lease build() { - return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds); + return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, + pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, + pendingCheckpointState, hashKeyRangeForLease); } } \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java new file mode 100644 index 00000000..02c71b03 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java @@ -0,0 +1,299 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; +import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.NullMetricsFactory; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LeaseCleanupManagerTest { + + private ShardInfo shardInfo; + private StreamIdentifier streamIdentifier; + private String concurrencyToken = "1234"; + + private String shardId = "shardId"; + private String splitParent = "splitParent"; + private String mergeParent1 = "mergeParent-1"; + private String mergeParent2 = "mergeParent-2"; + + private Duration maxFutureWait = Duration.ofSeconds(1); + private long leaseCleanupIntervalMillis = Duration.ofSeconds(1).toMillis(); + private long completedLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); + private long garbageLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); + private boolean cleanupLeasesOfCompletedShards = true; + private LeaseCleanupManager leaseCleanupManager; + private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory(); + + @Mock + private LeaseRefresher leaseRefresher; + @Mock + private LeaseCoordinator leaseCoordinator; + @Mock + private ShardDetector shardDetector; + @Mock + private ScheduledExecutorService deletionThreadPool; + + @Before + public void setUp() throws Exception { + shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + streamIdentifier = StreamIdentifier.singleStreamInstance("streamName"); + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + garbageLeaseCleanupIntervalMillis); + + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(any(Lease.class), any(UUID.class), any(String.class), any(String.class))).thenReturn(true); + } + + /** + * Tests subsequent calls to start {@link LeaseCleanupManager}. + */ + @Test + public final void testSubsequentStarts() { + leaseCleanupManager.start(); + Assert.assertTrue(leaseCleanupManager.isRunning()); + leaseCleanupManager.start(); + } + + /** + * Tests that when both child shard leases are present, we are able to delete the parent shard for the completed + * shard case. + */ + @Test + public final void testParentShardLeaseDeletedSplitCase() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 1); + } + + /** + * Tests that when both child shard leases are present, we are able to delete the parent shard for the completed + * shard case. + */ + @Test + public final void testParentShardLeaseDeletedMergeCase() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 1); + } + + /** + * Tests that if cleanupLeasesOfCompletedShards is not enabled by the customer, then no leases are cleaned up for + * the completed shard case. + */ + @Test + public final void testNoLeasesDeletedWhenNotEnabled() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + cleanupLeasesOfCompletedShards = false; + + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + garbageLeaseCleanupIntervalMillis); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 0); + } + + /** + * Tests that if some of the child shard leases are missing, we fail fast and don't delete the parent shard lease + * for the completed shard case. + */ + @Test + public final void testNoCleanupWhenSomeChildShardLeasesAreNotPresent() throws Exception { + List childShards = childShardsForSplit(); + + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, ExtendedSequenceNumber.LATEST, false, 0); + } + + /** + * Tests that if some child shard leases haven't begun processing (at least one lease w/ checkpoint TRIM_HORIZON), + * we don't delete them for the completed shard case. + */ + @Test + public final void testParentShardLeaseNotDeletedWhenChildIsAtTrim() throws Exception { + testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.TRIM_HORIZON); + } + + /** + * Tests that if some child shard leases haven't begun processing (at least one lease w/ checkpoint AT_TIMESTAMP), + * we don't delete them for the completed shard case. + */ + @Test + public final void testParentShardLeaseNotDeletedWhenChildIsAtTimestamp() throws Exception { + testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.AT_TIMESTAMP); + } + + private final void testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber extendedSequenceNumber) + throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), extendedSequenceNumber, 0); + } + + /** + * Tests that if a lease's parents are still present, we do not delete the lease. + */ + @Test + public final void testLeaseNotDeletedWhenParentsStillPresent() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.singleton("parent"), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 0); + } + + /** + * Tests ResourceNotFound case for if a shard expires, that we delete the lease when shardExpired is found. + */ + @Test + public final void testLeaseDeletedWhenShardDoesNotExist() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + + testLeaseDeletedWhenShardDoesNotExist(heldLease); + } + + /** + * Tests ResourceNotFound case when completed lease cleanup is disabled. + * @throws Exception + */ + @Test + public final void testLeaseDeletedWhenShardDoesNotExistAndCleanupCompletedLeaseDisabled() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + + cleanupLeasesOfCompletedShards = false; + + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + garbageLeaseCleanupIntervalMillis); + + testLeaseDeletedWhenShardDoesNotExist(heldLease); + } + + public void testLeaseDeletedWhenShardDoesNotExist(Lease heldLease) throws Exception { + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); + when(shardDetector.getChildShards(any(String.class))).thenThrow(ResourceNotFoundException.class); + when(leaseRefresher.getLease(heldLease.leaseKey())).thenReturn(heldLease); + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo, shardDetector)); + leaseCleanupManager.cleanupLeases(); + + verify(shardDetector, times(1)).getChildShards(shardInfo.shardId()); + verify(leaseRefresher, times(1)).deleteLease(heldLease); + } + + private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + ExtendedSequenceNumber extendedSequenceNumber, + int expectedDeletedLeases) throws Exception { + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, extendedSequenceNumber, true, expectedDeletedLeases); + } + + private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + ExtendedSequenceNumber extendedSequenceNumber, + boolean childShardLeasesPresent, + int expectedDeletedLeases) throws Exception { + + final Lease lease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", shardInfo.parentShardIds(), + childShards.stream().map(c -> c.shardId()).collect(Collectors.toSet())); + final List childShardLeases = childShards.stream().map(c -> LeaseHelper.createLease( + ShardInfo.getLeaseKey(shardInfo, c.shardId()), "leaseOwner", Collections.singleton(shardInfo.shardId()), + Collections.emptyList(), extendedSequenceNumber)).collect(Collectors.toList()); + + final List parentShardLeases = lease.parentShardIds().stream().map(p -> + LeaseHelper.createLease(ShardInfo.getLeaseKey(shardInfo, p), "leaseOwner", Collections.emptyList(), + Collections.singleton(shardInfo.shardId()), extendedSequenceNumber)).collect(Collectors.toList()); + + when(leaseRefresher.getLease(lease.leaseKey())).thenReturn(lease); + for (Lease parentShardLease : parentShardLeases) { + when(leaseRefresher.getLease(parentShardLease.leaseKey())).thenReturn(parentShardLease); + } + if (childShardLeasesPresent) { + for (Lease childShardLease : childShardLeases) { + when(leaseRefresher.getLease(childShardLease.leaseKey())).thenReturn(childShardLease); + } + } + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, lease, shardInfo, shardDetector)); + leaseCleanupManager.cleanupLeases(); + + verify(shardDetector, times(1)).getChildShards(shardInfo.shardId()); + verify(leaseRefresher, times(expectedDeletedLeases)).deleteLease(any(Lease.class)); + } + + private List childShardsForSplit() { + List parentShards = Arrays.asList(splitParent); + + ChildShard leftChild = ChildShard.builder() + .shardId("leftChild") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("rightChild") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + + return Arrays.asList(leftChild, rightChild); + } + + private List childShardsForMerge() { + List parentShards = Arrays.asList(mergeParent1, mergeParent2); + + ChildShard child = ChildShard.builder() + .shardId("onlyChild") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "99")) + .build(); + + return Collections.singletonList(child); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java new file mode 100644 index 00000000..0e10bc48 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java @@ -0,0 +1,44 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.Collection; +import java.util.Collections; + +public class LeaseHelper { + + public static Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + return createLease(leaseKey, leaseOwner, parentShardIds, Collections.emptySet(), ExtendedSequenceNumber.LATEST); + } + + public static Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds, Collection childShardIds) { + return createLease(leaseKey, leaseOwner, parentShardIds, childShardIds, ExtendedSequenceNumber.LATEST); + } + + public static Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds, + Collection childShardIds, ExtendedSequenceNumber extendedSequenceNumber) { + Lease lease = new Lease(); + lease.leaseKey(leaseKey); + lease.leaseOwner(leaseOwner); + lease.parentShardIds(parentShardIds); + lease.childShardIds(childShardIds); + lease.checkpoint(extendedSequenceNumber); + + return lease; + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java index f07a38f4..ee2504d8 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java @@ -68,7 +68,7 @@ public class ShardObjectHelper { String parentShardId, String adjacentParentShardId, SequenceNumberRange sequenceNumberRange) { - return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, null); + return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, HashKeyRange.builder().startingHashKey("1").endingHashKey("100").build()); } /** Helper method to create a new shard object. diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java index 42b826d1..ce6ce386 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java @@ -118,7 +118,8 @@ public class ShardSyncTaskIntegrationTest { leaseRefresher.deleteAll(); Set shardIds = shardDetector.listShards().stream().map(Shard::shardId).collect(Collectors.toSet()); ShardSyncTask syncTask = new ShardSyncTask(shardDetector, leaseRefresher, - InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), false, false, 0L, + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), + false, true, false, 0L, hierarchicalShardSyncer, NULL_METRICS_FACTORY); syncTask.call(); List leases = leaseRefresher.listLeases(); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java index 3af33c69..d89c010e 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java @@ -127,16 +127,37 @@ public class DynamoDBLeaseCoordinatorIntegrationTest { } assertNotNull(lease); - ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber("newCheckpoint"); + final ExtendedSequenceNumber initialCheckpoint = new ExtendedSequenceNumber("initialCheckpoint"); + final ExtendedSequenceNumber pendingCheckpoint = new ExtendedSequenceNumber("pendingCheckpoint"); + final ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber("newCheckpoint"); + final byte[] checkpointState = "checkpointState".getBytes(); + // lease's leaseCounter is wrong at this point, but it shouldn't matter. + assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), initialCheckpoint, lease.concurrencyToken())); + + final Lease leaseFromDDBAtInitialCheckpoint = leaseRefresher.getLease(lease.leaseKey()); + lease.leaseCounter(lease.leaseCounter() + 1); + lease.checkpoint(initialCheckpoint); + lease.leaseOwner(coordinator.workerIdentifier()); + assertEquals(lease, leaseFromDDBAtInitialCheckpoint); + + dynamoDBCheckpointer.prepareCheckpoint(lease.leaseKey(), pendingCheckpoint, lease.concurrencyToken().toString(), checkpointState); + + final Lease leaseFromDDBAtPendingCheckpoint = leaseRefresher.getLease(lease.leaseKey()); + lease.leaseCounter(lease.leaseCounter() + 1); + lease.checkpoint(initialCheckpoint); + lease.pendingCheckpoint(pendingCheckpoint); + lease.pendingCheckpointState(checkpointState); + assertEquals(lease, leaseFromDDBAtPendingCheckpoint); + assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), newCheckpoint, lease.concurrencyToken())); - Lease fromDynamo = leaseRefresher.getLease(lease.leaseKey()); - + final Lease leaseFromDDBAtNewCheckpoint = leaseRefresher.getLease(lease.leaseKey()); lease.leaseCounter(lease.leaseCounter() + 1); lease.checkpoint(newCheckpoint); - lease.leaseOwner(coordinator.workerIdentifier()); - assertEquals(lease, fromDynamo); + lease.pendingCheckpoint(null); + lease.pendingCheckpointState(null); + assertEquals(lease, leaseFromDDBAtNewCheckpoint); } /** diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java index 414f7975..1b2fa78a 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java @@ -14,6 +14,21 @@ */ package software.amazon.kinesis.leases.dynamodb; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.HashKeyRange; +import software.amazon.kinesis.common.HashKeyRangeForLease; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseIntegrationTest; +import software.amazon.kinesis.leases.UpdateField; +import software.amazon.kinesis.leases.exceptions.LeasingException; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -24,19 +39,6 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.verify; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.runners.MockitoJUnitRunner; - -import software.amazon.kinesis.leases.Lease; -import software.amazon.kinesis.leases.LeaseIntegrationTest; -import software.amazon.kinesis.leases.exceptions.LeasingException; - @RunWith(MockitoJUnitRunner.class) public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest { @@ -71,7 +73,7 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest Collection expected = builder.build().values(); // The / 3 here ensures that we will test Dynamo's paging mechanics. - List actual = leaseRefresher.list(numRecordsToPut / 3); + List actual = leaseRefresher.list(numRecordsToPut / 3, null); for (Lease lease : actual) { assertNotNull(expected.remove(lease)); @@ -101,6 +103,38 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest assertNull(actual); } + /** + * Tests leaseRefresher.updateLeaseWithMetaInfo() when the lease is deleted before updating it with meta info + */ + @Test + public void testDeleteLeaseThenUpdateLeaseWithMetaInfo() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + Lease lease = builder.withLease("1").build().get("1"); + final String leaseKey = lease.leaseKey(); + leaseRefresher.deleteLease(lease); + leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + final Lease deletedLease = leaseRefresher.getLease(leaseKey); + Assert.assertNull(deletedLease); + } + + /** + * Tests leaseRefresher.updateLeaseWithMetaInfo() on hashKeyRange update + */ + @Test + public void testUpdateLeaseWithMetaInfo() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + Lease lease = builder.withLease("1").build().get("1"); + final String leaseKey = lease.leaseKey(); + final HashKeyRangeForLease hashKeyRangeForLease = HashKeyRangeForLease.fromHashKeyRange(HashKeyRange.builder() + .startingHashKey("1") + .endingHashKey("2") + .build()); + lease.hashKeyRange(hashKeyRangeForLease); + leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + final Lease updatedLease = leaseRefresher.getLease(leaseKey); + Assert.assertEquals(lease, updatedLease); + } + /** * Tests leaseRefresher.holdLease's success scenario. */ @@ -239,6 +273,18 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest assertNull(newLease); } + @Test + public void testUpdateLease() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + Lease lease = builder.withLease("1").build().get("1"); + Lease updatedLease = lease.copy(); + updatedLease.childShardIds(Collections.singleton("updatedChildShardId")); + + leaseRefresher.updateLease(updatedLease); + Lease newLease = leaseRefresher.getLease(lease.leaseKey()); + assertEquals(Collections.singleton("updatedChildShardId"), newLease.childShardIds()); + } + /** * Tests deleteLease when a lease does not exist. */ diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java index 61cba722..bfff4e92 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java @@ -37,6 +37,7 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.exceptions.DependencyException; @@ -55,7 +56,8 @@ public class DynamoDBLeaseRenewerTest { private LeaseRefresher leaseRefresher; private static Lease newLease(String leaseKey) { - return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>()); + return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, + new HashSet<>(), new HashSet<>(), null, HashKeyRangeForLease.deserialize("1", "2")); } @Before diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java index 2e3d65a4..972d3951 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java @@ -16,22 +16,17 @@ package software.amazon.kinesis.leases.dynamodb; import java.util.Collection; import java.util.Map; - import org.junit.Assert; import org.junit.Before; import org.junit.Test; - import org.junit.runner.RunWith; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseIntegrationTest; import software.amazon.kinesis.leases.exceptions.LeasingException; import software.amazon.kinesis.metrics.NullMetricsFactory; - import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; @RunWith(MockitoJUnitRunner.class) public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest { @@ -105,9 +100,30 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest { builder.withLease("4", "bar").build(); + // setting multiplier to unusually high number to avoid very old lease taking + taker.withVeryOldLeaseDurationNanosMultipler(5000000000L); builder.takeMutateAssert(taker, 2); } + /** + * Verify that we take all very old leases by setting up an environment where there are 4 leases and 2 workers, + * only one of which holds a lease. This leaves 3 free leases. LeaseTaker should take all 3 leases since they + * are denoted as very old. + */ + @Test + public void testVeryOldLeaseTaker() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + + for (int i = 0; i < 3; i++) { + builder.withLease(Integer.toString(i), null); + } + + builder.withLease("4", "bar").build(); + + // setting multiplier to unusually high number to avoid very old lease taking + builder.takeMutateAssert(taker, 3); + } + /** * Verify that when getAllLeases() is called, DynamoDBLeaseTaker * - does not call listLeases() diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java index 6afa0045..677303d6 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertNotNull; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; @@ -72,6 +73,7 @@ public class TestHarnessBuilder { lease.leaseCounter(0L); lease.leaseOwner(owner); lease.parentShardIds(Collections.singleton("parentShardId")); + lease.childShardIds(new HashSet<>()); lease.leaseKey(shardId); return lease; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java index a1cd95d7..06a72230 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java @@ -22,6 +22,7 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import org.junit.Before; import org.junit.Test; @@ -40,6 +41,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; public class BlockOnParentShardTaskTest { private final long backoffTimeInMillis = 50L; private final String shardId = "shardId-97"; + private final String streamId = "123:stream:146"; private final String concurrencyToken = "testToken"; private final List emptyParentShardIds = new ArrayList(); private ShardInfo shardInfo; @@ -73,7 +75,7 @@ public class BlockOnParentShardTaskTest { * @throws DependencyException */ @Test - public final void testCallWhenParentsHaveFinished() + public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinished() throws DependencyException, InvalidStateException, ProvisionedThroughputException { ShardInfo shardInfo = null; @@ -107,6 +109,50 @@ public class BlockOnParentShardTaskTest { assertNull(result.getException()); } + /** + * Test call() when there are 1-2 parent shards that have been fully processed. + * @throws ProvisionedThroughputException + * @throws InvalidStateException + * @throws DependencyException + */ + @Test + public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinishedMultiStream() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + + ShardInfo shardInfo = null; + BlockOnParentShardTask task = null; + String parent1LeaseKey = streamId + ":" + "shardId-1"; + String parent2LeaseKey = streamId + ":" + "shardId-2"; + String parent1ShardId = "shardId-1"; + String parent2ShardId = "shardId-2"; + List parentShardIds = new ArrayList<>(); + TaskResult result = null; + + Lease parent1Lease = new Lease(); + parent1Lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + Lease parent2Lease = new Lease(); + parent2Lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + + LeaseRefresher leaseRefresher = mock(LeaseRefresher.class); + when(leaseRefresher.getLease(parent1LeaseKey)).thenReturn(parent1Lease); + when(leaseRefresher.getLease(parent2LeaseKey)).thenReturn(parent2Lease); + + // test single parent + parentShardIds.add(parent1ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, + streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNull(result.getException()); + + // test two parents + parentShardIds.add(parent2ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNull(result.getException()); + } + /** * Test call() when there are 1-2 parent shards that have NOT been fully processed. * @throws ProvisionedThroughputException @@ -115,7 +161,7 @@ public class BlockOnParentShardTaskTest { */ @Test public final void testCallWhenParentsHaveNotFinished() - throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throws DependencyException, InvalidStateException, ProvisionedThroughputException { ShardInfo shardInfo = null; BlockOnParentShardTask task = null; @@ -149,6 +195,50 @@ public class BlockOnParentShardTaskTest { assertNotNull(result.getException()); } + /** + * Test call() when there are 1-2 parent shards that have NOT been fully processed. + * @throws ProvisionedThroughputException + * @throws InvalidStateException + * @throws DependencyException + */ + @Test + public final void testCallWhenParentsHaveNotFinishedMultiStream() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + + ShardInfo shardInfo = null; + BlockOnParentShardTask task = null; + String parent1LeaseKey = streamId + ":" + "shardId-1"; + String parent2LeaseKey = streamId + ":" + "shardId-2"; + String parent1ShardId = "shardId-1"; + String parent2ShardId = "shardId-2"; + List parentShardIds = new ArrayList<>(); + TaskResult result = null; + + Lease parent1Lease = new Lease(); + parent1Lease.checkpoint(ExtendedSequenceNumber.LATEST); + Lease parent2Lease = new Lease(); + // mock a sequence number checkpoint + parent2Lease.checkpoint(new ExtendedSequenceNumber("98182584034")); + + LeaseRefresher leaseRefresher = mock(LeaseRefresher.class); + when(leaseRefresher.getLease(parent1LeaseKey)).thenReturn(parent1Lease); + when(leaseRefresher.getLease(parent2LeaseKey)).thenReturn(parent2Lease); + + // test single parent + parentShardIds.add(parent1ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNotNull(result.getException()); + + // test two parents + parentShardIds.add(parent2ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNotNull(result.getException()); + } + /** * Test call() with 1 parent shard before and after it is completely processed. * @throws ProvisionedThroughputException diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java index 16f5e9a4..235937d0 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java @@ -25,6 +25,8 @@ import static org.mockito.Mockito.when; import static software.amazon.kinesis.lifecycle.ConsumerStates.ShardConsumerState; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutorService; @@ -33,21 +35,24 @@ import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -56,7 +61,6 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; -import javax.swing.*; @RunWith(MockitoJUnitRunner.class) public class ConsumerStatesTest { @@ -99,6 +103,8 @@ public class ConsumerStatesTest { private ProcessRecordsInput processRecordsInput; @Mock private TaskExecutionListener taskExecutionListener; + @Mock + private LeaseCleanupManager leaseCleanupManager; private long parentShardPollIntervalMillis = 0xCAFE; private boolean cleanupLeasesOfCompletedShards = true; @@ -114,16 +120,16 @@ public class ConsumerStatesTest { @Before public void setup() { - argument = new ShardConsumerArgument(shardInfo, STREAM_NAME, leaseCoordinator, executorService, recordsPublisher, + argument = new ShardConsumerArgument(shardInfo, StreamIdentifier.singleStreamInstance(STREAM_NAME), leaseCoordinator, executorService, recordsPublisher, shardRecordProcessor, checkpointer, recordProcessorCheckpointer, parentShardPollIntervalMillis, taskBackoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, shouldCallProcessRecordsEvenForEmptyRecordList, idleTimeInMillis, INITIAL_POSITION_IN_STREAM, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, shardDetector, - new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory); + new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory, leaseCleanupManager); + when(shardInfo.shardId()).thenReturn("shardId-000000000000"); + when(shardInfo.streamIdentifierSerOpt()).thenReturn(Optional.of(StreamIdentifier.singleStreamInstance(STREAM_NAME).serialize())); consumer = spy(new ShardConsumer(recordsPublisher, executorService, shardInfo, logWarningForTaskAfterMillis, argument, taskExecutionListener, 0)); - - when(shardInfo.shardId()).thenReturn("shardId-000000000000"); when(recordProcessorCheckpointer.checkpointer()).thenReturn(checkpointer); } @@ -145,7 +151,7 @@ public class ConsumerStatesTest { assertThat(state.successTransition(), equalTo(ShardConsumerState.INITIALIZING.consumerState())); for (ShutdownReason shutdownReason : ShutdownReason.values()) { assertThat(state.shutdownTransition(shutdownReason), - equalTo(ShardConsumerState.SHUTDOWN_COMPLETE.consumerState())); + equalTo(ShardConsumerState.SHUTTING_DOWN.consumerState())); } assertThat(state.state(), equalTo(ShardConsumerState.WAITING_ON_PARENT_SHARDS)); @@ -301,13 +307,27 @@ public class ConsumerStatesTest { } - // TODO: Fix this test - @Ignore @Test public void shuttingDownStateTest() { consumer.markForShutdown(ShutdownReason.SHARD_END); ConsumerState state = ShardConsumerState.SHUTTING_DOWN.consumerState(); - ConsumerTask task = state.createTask(argument, consumer, null); + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add("shardId-000000000000"); + ChildShard leftChild = ChildShard.builder() + .shardId("shardId-000000000001") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("shardId-000000000002") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + when(processRecordsInput.childShards()).thenReturn(childShards); + ConsumerTask task = state.createTask(argument, consumer, processRecordsInput); assertThat(task, shutdownTask(ShardInfo.class, "shardInfo", equalTo(shardInfo))); assertThat(task, @@ -316,8 +336,6 @@ public class ConsumerStatesTest { equalTo(recordProcessorCheckpointer))); assertThat(task, shutdownTask(ShutdownReason.class, "reason", equalTo(reason))); assertThat(task, shutdownTask(LeaseCoordinator.class, "leaseCoordinator", equalTo(leaseCoordinator))); - assertThat(task, shutdownTask(InitialPositionInStreamExtended.class, "initialPositionInStream", - equalTo(initialPositionInStream))); assertThat(task, shutdownTask(Boolean.class, "cleanupLeasesOfCompletedShards", equalTo(cleanupLeasesOfCompletedShards))); assertThat(task, shutdownTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java index 6af62edb..688bd199 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java @@ -17,37 +17,53 @@ package software.amazon.kinesis.lifecycle; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doAnswer; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static software.amazon.kinesis.lifecycle.ShutdownTask.RETRY_RANDOM_MAX_RANGE; -import java.util.Arrays; +import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.UUID; +import com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.Matchers; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; -import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; +import software.amazon.kinesis.leases.LeaseHelper; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardObjectHelper; +import software.amazon.kinesis.leases.UpdateField; +import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; import software.amazon.kinesis.metrics.MetricsFactory; @@ -69,12 +85,13 @@ public class ShutdownTaskTest { private static final ShutdownReason LEASE_LOST_SHUTDOWN_REASON = ShutdownReason.LEASE_LOST; private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory(); - private final String concurrencyToken = "testToken4398"; + private final String concurrencyToken = "0-1-2-3-4"; private final String shardId = "shardId-0"; private boolean cleanupLeasesOfCompletedShards = false; private boolean ignoreUnexpectedChildShards = false; private ShardInfo shardInfo; private ShutdownTask task; + private StreamIdentifier streamIdentifier = StreamIdentifier.singleStreamInstance("streamName"); @Mock private RecordsPublisher recordsPublisher; @@ -92,11 +109,17 @@ public class ShutdownTaskTest { private HierarchicalShardSyncer hierarchicalShardSyncer; @Mock private ShardRecordProcessor shardRecordProcessor; + @Mock + private LeaseCleanupManager leaseCleanupManager; @Before public void setUp() throws Exception { doNothing().when(recordsPublisher).shutdown(); when(recordProcessorCheckpointer.checkpointer()).thenReturn(checkpointer); + final Lease childLease = new Lease(); + childLease.leaseKey("childShardLeaseKey"); + when(hierarchicalShardSyncer.createLeaseForChildShard(Matchers.any(ChildShard.class), Matchers.any(StreamIdentifier.class))) + .thenReturn(childLease); shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); @@ -104,7 +127,7 @@ public class ShutdownTaskTest { task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards(), streamIdentifier, leaseCleanupManager); } /** @@ -112,13 +135,17 @@ public class ShutdownTaskTest { * This test is for the scenario that customer doesn't implement checkpoint in their implementation */ @Test - public final void testCallWhenApplicationDoesNotCheckpoint() { - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); + public final void testCallWhenApplicationDoesNotCheckpoint() throws Exception { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); + Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId"), Collections.emptyList(), ExtendedSequenceNumber.LATEST); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); + when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true); final TaskResult result = task.call(); assertNotNull(result.getException()); - assertTrue(result.getException() instanceof IllegalArgumentException); + assertTrue(result.getException() instanceof CustomerApplicationException); } /** @@ -126,25 +153,20 @@ public class ShutdownTaskTest { * This test is for the scenario that checkAndCreateLeaseForNewShards throws an exception. */ @Test - public final void testCallWhenSyncingShardsThrows() throws Exception { - List latestShards = constructShardListGraphA(); - when(shardDetector.listShards()).thenReturn(latestShards); + public final void testCallWhenCreatingNewLeasesThrows() throws Exception { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId")); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); - - doAnswer((invocation) -> { - throw new KinesisClientLibIOException("KinesisClientLibIOException"); - }).when(hierarchicalShardSyncer) - .checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, INITIAL_POSITION_TRIM_HORIZON, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, - NULL_METRICS_FACTORY.createMetrics(), latestShards); + when(hierarchicalShardSyncer.createLeaseForChildShard(Matchers.any(ChildShard.class), Matchers.any(StreamIdentifier.class))) + .thenThrow(new InvalidStateException("InvalidStateException is thrown")); final TaskResult result = task.call(); - assertNotNull(result.getException()); - assertTrue(result.getException() instanceof KinesisClientLibIOException); + assertNull(result.getException()); verify(recordsPublisher).shutdown(); - verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator).dropLease(Matchers.any(Lease.class)); } /** @@ -152,24 +174,147 @@ public class ShutdownTaskTest { * This test is for the scenario that ShutdownTask is created for ShardConsumer reaching the Shard End. */ @Test - public final void testCallWhenTrueShardEnd() { + public final void testCallWhenTrueShardEnd() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards(), streamIdentifier, leaseCleanupManager); - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId")); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true); + when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); - verify(shardDetector, times(1)).listShards(); - verify(leaseCoordinator, never()).getAssignments(); + verify(leaseRefresher).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class)); + verify(leaseRefresher, times(2)).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class)); + } + + @Test + public final void testCallThrowsUntilParentInfoNotPresentInLease() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2")); + Lease parentLease = LeaseHelper.createLease("shardId-1", "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-1")) + .thenReturn(null, null, null, null, null, parentLease); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true); + when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease); + // Return null lease first time to simulate partial parent lease info + when(leaseRefresher.getLease("shardId-1")) + .thenReturn(null, null, null, null, null, parentLease); + + // Make first 5 attempts with partial parent info in lease table + for (int i = 0; i < 5; i++) { + ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, + SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager)); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); + TaskResult result = task.call(); + assertNotNull(result.getException()); + assertTrue(result.getException() instanceof BlockedOnParentShardException); + assertTrue(result.getException().getMessage().contains("has partial parent information in lease table")); + verify(recordsPublisher, never()).shutdown(); + verify(shardRecordProcessor, never()) + .shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator, never()) + .updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); + verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); + } + + // make next attempt with complete parent info in lease table + ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, + SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager)); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); + TaskResult result = task.call(); + assertNull(result.getException()); + verify(recordsPublisher).shutdown(); + verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseRefresher).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class)); + verify(leaseRefresher, times(1)).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(task, never()).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class)); + } + + @Test + public final void testCallTriggersLeaseLossWhenParentInfoNotPresentInLease() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2")); + Lease parentLease = LeaseHelper.createLease("shardId-1", "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-1")) + .thenReturn(null, null, null, null, null, null, null, null, null, null, null); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true); + when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease); + // Return null lease first time to simulate partial parent lease info + when(leaseRefresher.getLease("shardId-1")) + .thenReturn(null, null, null, null, null, null, null, null, null, null, null); + + // Make first 10 attempts with partial parent info in lease table + for (int i = 0; i < 10; i++) { + ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, + SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager)); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); + TaskResult result = task.call(); + assertNotNull(result.getException()); + assertTrue(result.getException() instanceof BlockedOnParentShardException); + assertTrue(result.getException().getMessage().contains("has partial parent information in lease table")); + verify(recordsPublisher, never()).shutdown(); + verify(shardRecordProcessor, never()) + .shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator, never()) + .updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); + verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); + } + + // make final attempt with incomplete parent info in lease table + ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, + SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager)); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(true); + TaskResult result = task.call(); + assertNull(result.getException()); + verify(recordsPublisher).shutdown(); + verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); + verify(leaseRefresher, never()).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class)); + verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); + verify(leaseCoordinator).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); } /** @@ -177,23 +322,26 @@ public class ShutdownTaskTest { * This test is for the scenario that a ShutdownTask is created for detecting a false Shard End. */ @Test - public final void testCallWhenFalseShardEnd() { + public final void testCallWhenShardNotFound() throws Exception { + final Lease heldLease = LeaseHelper.createLease("shardId-4", "leaseOwner", Collections.emptyList()); shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>(), streamIdentifier, leaseCleanupManager); - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); + when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseRefresher.getLease("shardId-4")).thenReturn(heldLease); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-4")).thenReturn(heldLease); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); - verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); - verify(shardDetector, times(1)).listShards(); - verify(leaseCoordinator).getCurrentlyHeldLease(shardInfo.shardId()); + verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } /** @@ -201,25 +349,23 @@ public class ShutdownTaskTest { * This test is for the scenario that a ShutdownTask is created for the ShardConsumer losing the lease. */ @Test - public final void testCallWhenLeaseLost() { + public final void testCallWhenLeaseLost() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); + ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, - LEASE_LOST_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, - ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); - - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); + LEASE_LOST_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>(), streamIdentifier, leaseCleanupManager); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); - verify(shardDetector, never()).listShards(); verify(leaseCoordinator, never()).getAssignments(); + verify(leaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(any(Lease.class)); } - /** * Test method for {@link ShutdownTask#taskType()}. */ @@ -228,45 +374,36 @@ public class ShutdownTaskTest { assertEquals(TaskType.SHUTDOWN, task.taskType()); } - - /* - * 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 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - */ - private List constructShardListGraphA() { - final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); - final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); - final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("11", "205"); - 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", "99")), - ShardObjectHelper.newShard("shardId-1", null, null, range0, - ShardObjectHelper.newHashKeyRange("100", "199")), - ShardObjectHelper.newShard("shardId-2", null, null, range0, - ShardObjectHelper.newHashKeyRange("200", "299")), - ShardObjectHelper.newShard("shardId-3", null, null, range0, - ShardObjectHelper.newHashKeyRange("300", "399")), - ShardObjectHelper.newShard("shardId-4", null, null, range1, - ShardObjectHelper.newHashKeyRange("400", "499")), - ShardObjectHelper.newShard("shardId-5", null, null, range2, - ShardObjectHelper.newHashKeyRange("500", ShardObjectHelper.MAX_HASH_KEY)), - ShardObjectHelper.newShard("shardId-6", "shardId-0", "shardId-1", range3, - ShardObjectHelper.newHashKeyRange("0", "199")), - ShardObjectHelper.newShard("shardId-7", "shardId-2", "shardId-3", range3, - ShardObjectHelper.newHashKeyRange("200", "399")), - ShardObjectHelper.newShard("shardId-8", "shardId-6", "shardId-7", range4, - ShardObjectHelper.newHashKeyRange("0", "399")), - ShardObjectHelper.newShard("shardId-9", "shardId-5", null, range4, - ShardObjectHelper.newHashKeyRange("500", "799")), - ShardObjectHelper.newShard("shardId-10", null, "shardId-5", range4, - ShardObjectHelper.newHashKeyRange("800", ShardObjectHelper.MAX_HASH_KEY))); + private List constructChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(shardId); + ChildShard leftChild = ChildShard.builder() + .shardId("ShardId-1") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("ShardId-2") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + return childShards; } + private List constructChildShard() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(shardId); + parentShards.add("shardId-1"); + ChildShard leftChild = ChildShard.builder() + .shardId("shardId-2") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + childShards.add(leftChild); + return childShards; + } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java index a0d18d56..4fee3d08 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java @@ -19,19 +19,27 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.nullValue; import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.RetrievalFactory; +import java.util.Optional; + @RunWith(MockitoJUnitRunner.class) public class FanOutConfigTest { @@ -44,6 +52,13 @@ public class FanOutConfigTest { private FanOutConsumerRegistration consumerRegistration; @Mock private KinesisAsyncClient kinesisClient; + @Mock + private StreamConfig streamConfig; + + @Before + public void setup() { + when(streamConfig.consumerArn()).thenReturn(null); + } @Test public void testNoRegisterIfConsumerArnSet() throws Exception { @@ -59,7 +74,35 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) .streamName(TEST_STREAM_NAME); RetrievalFactory retrievalFactory = config.retrievalFactory(); + ShardInfo shardInfo = mock(ShardInfo.class); +// doReturn(Optional.of(StreamIdentifier.singleStreamInstance(TEST_STREAM_NAME).serialize())).when(shardInfo).streamIdentifier(); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); + retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); + assertThat(retrievalFactory, not(nullValue())); + verify(consumerRegistration).getOrCreateStreamConsumerArn(); + } + @Test + public void testRegisterNotCalledWhenConsumerArnSetInMultiStreamMode() throws Exception { + when(streamConfig.consumerArn()).thenReturn("consumerArn"); + FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) + .streamName(TEST_STREAM_NAME); + RetrievalFactory retrievalFactory = config.retrievalFactory(); + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of("account:stream:12345")).when(shardInfo).streamIdentifierSerOpt(); + retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); + assertThat(retrievalFactory, not(nullValue())); + verify(consumerRegistration, never()).getOrCreateStreamConsumerArn(); + } + + @Test + public void testRegisterCalledWhenConsumerArnNotSetInMultiStreamMode() throws Exception { + FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) + .streamName(TEST_STREAM_NAME); + RetrievalFactory retrievalFactory = config.retrievalFactory(); + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of("account:stream:12345")).when(shardInfo).streamIdentifierSerOpt(); + retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(retrievalFactory, not(nullValue())); verify(consumerRegistration).getOrCreateStreamConsumerArn(); } @@ -83,7 +126,9 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) .streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; @@ -96,9 +141,10 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).consumerName(TEST_CONSUMER_NAME) .streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); - TestingConfig testingConfig = (TestingConfig) config; assertThat(testingConfig.stream, equalTo(TEST_STREAM_NAME)); assertThat(testingConfig.consumerToCreate, equalTo(TEST_CONSUMER_NAME)); @@ -109,7 +155,9 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) .consumerName(TEST_CONSUMER_NAME).streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java index fe6489b9..43881122 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java @@ -26,6 +26,7 @@ import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.core.async.SdkPublisher; import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; @@ -35,6 +36,7 @@ import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.lifecycle.ShardConsumerNotifyingSubscriber; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.retrieval.BatchUniqueIdentifier; @@ -47,6 +49,7 @@ import software.amazon.kinesis.utils.SubscribeToShardRequestMatcher; import java.nio.ByteBuffer; import java.time.Instant; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.Optional; @@ -89,6 +92,7 @@ public class FanOutRecordsPublisherTest { private static final String SHARD_ID = "Shard-001"; private static final String CONSUMER_ARN = "arn:consumer"; + private static final String CONTINUATION_SEQUENCE_NUMBER = "continuationSequenceNumber"; @Mock private KinesisAsyncClient kinesisClient; @@ -148,7 +152,12 @@ public class FanOutRecordsPublisherTest { List matchers = records.stream().map(KinesisClientRecordMatcher::new) .collect(Collectors.toList()); - batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).build(); + batchEvent = SubscribeToShardEvent.builder() + .millisBehindLatest(100L) + .records(records) + .continuationSequenceNumber("test") + .childShards(Collections.emptyList()) + .build(); captor.getValue().onNext(batchEvent); captor.getValue().onNext(batchEvent); @@ -166,6 +175,73 @@ public class FanOutRecordsPublisherTest { } + @Test + public void InvalidEventTest() throws Exception { + FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN); + + ArgumentCaptor captor = ArgumentCaptor + .forClass(FanOutRecordsPublisher.RecordSubscription.class); + ArgumentCaptor flowCaptor = ArgumentCaptor + .forClass(FanOutRecordsPublisher.RecordFlow.class); + + doNothing().when(publisher).subscribe(captor.capture()); + + source.start(ExtendedSequenceNumber.LATEST, + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)); + + List receivedInput = new ArrayList<>(); + + source.subscribe(new ShardConsumerNotifyingSubscriber(new Subscriber() { + Subscription subscription; + + @Override public void onSubscribe(Subscription s) { + subscription = s; + subscription.request(1); + } + + @Override public void onNext(RecordsRetrieved input) { + receivedInput.add(input.processRecordsInput()); + subscription.request(1); + } + + @Override public void onError(Throwable t) { + log.error("Caught throwable in subscriber", t); + fail("Caught throwable in subscriber"); + } + + @Override public void onComplete() { + fail("OnComplete called when not expected"); + } + }, source)); + + verify(kinesisClient).subscribeToShard(any(SubscribeToShardRequest.class), flowCaptor.capture()); + flowCaptor.getValue().onEventStream(publisher); + captor.getValue().onSubscribe(subscription); + + List records = Stream.of(1, 2, 3).map(this::makeRecord).collect(Collectors.toList()); + List matchers = records.stream().map(KinesisClientRecordMatcher::new) + .collect(Collectors.toList()); + + batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).continuationSequenceNumber(CONTINUATION_SEQUENCE_NUMBER).build(); + SubscribeToShardEvent invalidEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).childShards(Collections.emptyList()).build(); + + captor.getValue().onNext(batchEvent); + captor.getValue().onNext(invalidEvent); + captor.getValue().onNext(batchEvent); + + // When the second request failed with invalid event, it should stop sending requests and cancel the flow. + verify(subscription, times(2)).request(1); + assertThat(receivedInput.size(), equalTo(1)); + + receivedInput.stream().map(ProcessRecordsInput::records).forEach(clientRecordsList -> { + assertThat(clientRecordsList.size(), equalTo(matchers.size())); + for (int i = 0; i < clientRecordsList.size(); ++i) { + assertThat(clientRecordsList.get(i), matchers.get(i)); + } + }); + + } + @Test public void testIfAllEventsReceivedWhenNoTasksRejectedByExecutor() throws Exception { FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN); @@ -225,7 +301,9 @@ public class FanOutRecordsPublisherTest { SubscribeToShardEvent.builder() .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum) - .records(records).build()) + .records(records) + .childShards(Collections.emptyList()) + .build()) .forEach(batchEvent -> captor.getValue().onNext(batchEvent)); verify(subscription, times(4)).request(1); @@ -301,7 +379,9 @@ public class FanOutRecordsPublisherTest { SubscribeToShardEvent.builder() .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum) - .records(records).build()) + .records(records) + .childShards(Collections.emptyList()) + .build()) .forEach(batchEvent -> captor.getValue().onNext(batchEvent)); verify(subscription, times(2)).request(1); @@ -334,6 +414,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -436,6 +517,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -536,13 +618,30 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(SHARD_ID); + ChildShard leftChild = ChildShard.builder() + .shardId("Shard-002") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("Shard-003") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); Consumer servicePublisherShardEndAction = contSeqNum -> captor.getValue().onNext( SubscribeToShardEvent.builder() .millisBehindLatest(100L) .continuationSequenceNumber(null) .records(records) + .childShards(childShards) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -648,6 +747,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -750,6 +850,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -842,6 +943,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(1); @@ -1004,7 +1106,12 @@ public class FanOutRecordsPublisherTest { List matchers = records.stream().map(KinesisClientRecordMatcher::new) .collect(Collectors.toList()); - batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).build(); + batchEvent = SubscribeToShardEvent.builder() + .millisBehindLatest(100L) + .records(records) + .continuationSequenceNumber(CONTINUATION_SEQUENCE_NUMBER) + .childShards(Collections.emptyList()) + .build(); captor.getValue().onNext(batchEvent); captor.getValue().onNext(batchEvent); @@ -1098,7 +1205,7 @@ public class FanOutRecordsPublisherTest { .collect(Collectors.toList()); batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records) - .continuationSequenceNumber("3").build(); + .continuationSequenceNumber("3").childShards(Collections.emptyList()).build(); captor.getValue().onNext(batchEvent); captor.getValue().onComplete(); @@ -1126,7 +1233,7 @@ public class FanOutRecordsPublisherTest { .collect(Collectors.toList()); batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(nextRecords) - .continuationSequenceNumber("6").build(); + .continuationSequenceNumber("6").childShards(Collections.emptyList()).build(); nextSubscribeCaptor.getValue().onNext(batchEvent); verify(subscription, times(4)).request(1); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java index a88f3c3b..74b0c125 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java @@ -30,6 +30,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -53,6 +54,7 @@ import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; @@ -65,6 +67,7 @@ import software.amazon.kinesis.checkpoint.SentinelCheckpoint; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.exceptions.KinesisClientLibException; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.NullMetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -330,8 +333,31 @@ public class KinesisDataFetcherTest { private CompletableFuture makeGetRecordsResponse(String nextIterator, List records) throws InterruptedException, ExecutionException { + List childShards = new ArrayList<>(); + if(nextIterator == null) { + childShards = createChildShards(); + } return CompletableFuture.completedFuture(GetRecordsResponse.builder().nextShardIterator(nextIterator) - .records(CollectionUtils.isNullOrEmpty(records) ? Collections.emptyList() : records).build()); + .records(CollectionUtils.isNullOrEmpty(records) ? Collections.emptyList() : records).childShards(childShards).build()); + } + + private List createChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(SHARD_ID); + ChildShard leftChild = ChildShard.builder() + .shardId("Shard-2") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("Shard-3") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + return childShards; } @Test @@ -342,6 +368,7 @@ public class KinesisDataFetcherTest { final String initialIterator = "InitialIterator"; final String nextIterator1 = "NextIteratorOne"; final String nextIterator2 = "NextIteratorTwo"; + final String nextIterator3 = "NextIteratorThree"; final CompletableFuture nonAdvancingResult1 = makeGetRecordsResponse(initialIterator, null); final CompletableFuture nonAdvancingResult2 = makeGetRecordsResponse(nextIterator1, null); final CompletableFuture finalNonAdvancingResult = makeGetRecordsResponse(nextIterator2, diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java index f940faf2..461fce71 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -48,6 +49,7 @@ import org.mockito.runners.MockitoJUnitRunner; import org.mockito.stubbing.Answer; import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import lombok.extern.slf4j.Slf4j; @@ -86,6 +88,7 @@ public class PrefetchRecordsPublisherIntegrationTest { private String operation = "ProcessTask"; private String streamName = "streamName"; private String shardId = "shardId-000000000000"; + private String nextShardIterator = "testNextShardIterator"; @Mock private KinesisAsyncClient kinesisClient; @@ -249,7 +252,7 @@ public class PrefetchRecordsPublisherIntegrationTest { @Override public DataFetcherResult getRecords() { - GetRecordsResponse getRecordsResult = GetRecordsResponse.builder().records(new ArrayList<>(records)).millisBehindLatest(1000L).build(); + GetRecordsResponse getRecordsResult = GetRecordsResponse.builder().records(new ArrayList<>(records)).nextShardIterator(nextShardIterator).millisBehindLatest(1000L).build(); return new AdvancingResult(getRecordsResult); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java index f5772aaf..f12e2310 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -30,10 +31,12 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static software.amazon.kinesis.utils.BlockingUtils.blockUntilConditionSatisfied; import static software.amazon.kinesis.utils.BlockingUtils.blockUntilRecordsAvailable; import static software.amazon.kinesis.utils.ProcessRecordsInputMatcher.eqProcessRecordsInput; @@ -46,6 +49,7 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -56,6 +60,7 @@ import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; @@ -71,11 +76,14 @@ import io.reactivex.Flowable; import io.reactivex.schedulers.Schedulers; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.leases.ShardObjectHelper; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.lifecycle.ShardConsumerNotifyingSubscriber; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.metrics.NullMetricsFactory; @@ -98,11 +106,12 @@ public class PrefetchRecordsPublisherTest { private static final int MAX_SIZE = 5; private static final int MAX_RECORDS_COUNT = 15000; private static final long IDLE_MILLIS_BETWEEN_CALLS = 0L; + private static final String NEXT_SHARD_ITERATOR = "testNextShardIterator"; @Mock private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @Mock - private KinesisDataFetcher dataFetcher; + private DataFetcher dataFetcher; @Mock private InitialPositionInStreamExtended initialPosition; @Mock @@ -119,8 +128,8 @@ public class PrefetchRecordsPublisherTest { @Before public void setup() { - when(getRecordsRetrievalStrategy.getDataFetcher()).thenReturn(dataFetcher); - + when(getRecordsRetrievalStrategy.dataFetcher()).thenReturn(dataFetcher); + when(dataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("testStream")); executorService = spy(Executors.newFixedThreadPool(1)); getRecordsCache = new PrefetchRecordsPublisher( MAX_SIZE, @@ -135,11 +144,50 @@ public class PrefetchRecordsPublisherTest { "shardId"); spyQueue = spy(getRecordsCache.getPublisherSession().prefetchRecordsQueue()); records = spy(new ArrayList<>()); - getRecordsResponse = GetRecordsResponse.builder().records(records).build(); + getRecordsResponse = GetRecordsResponse.builder().records(records).nextShardIterator(NEXT_SHARD_ITERATOR).childShards(new ArrayList<>()).build(); when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResponse); } + @Test + public void testDataFetcherIsNotReInitializedOnMultipleCacheStarts() { + getRecordsCache.start(sequenceNumber, initialPosition); + getRecordsCache.start(sequenceNumber, initialPosition); + getRecordsCache.start(sequenceNumber, initialPosition); + verify(dataFetcher, times(1)).initialize(any(ExtendedSequenceNumber.class), any()); + } + + @Test + public void testPrefetchPublisherInternalStateNotModifiedWhenPrefetcherThreadStartFails() { + doThrow(new RejectedExecutionException()).doThrow(new RejectedExecutionException()).doCallRealMethod() + .when(executorService).execute(any()); + // Initialize try 1 + tryPrefetchCacheStart(); + blockUntilConditionSatisfied(() -> getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == MAX_SIZE, 300); + verifyInternalState(0); + // Initialize try 2 + tryPrefetchCacheStart(); + blockUntilConditionSatisfied(() -> getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == MAX_SIZE, 300); + verifyInternalState(0); + // Initialize try 3 + tryPrefetchCacheStart(); + blockUntilConditionSatisfied(() -> getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == MAX_SIZE, 300); + verifyInternalState(MAX_SIZE); + verify(dataFetcher, times(3)).initialize(any(ExtendedSequenceNumber.class), any()); + } + + private void tryPrefetchCacheStart() { + try { + getRecordsCache.start(sequenceNumber, initialPosition); + } catch (Exception e) { + // suppress exception + } + } + + private void verifyInternalState(int queueSize) { + Assert.assertTrue(getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == queueSize); + } + @Test public void testGetRecords() { record = Record.builder().data(createByteBufferWithSize(SIZE_512_KB)).build(); @@ -154,11 +202,67 @@ public class PrefetchRecordsPublisherTest { .processRecordsInput(); assertEquals(expectedRecords, result.records()); + assertEquals(new ArrayList<>(), result.childShards()); verify(executorService).execute(any()); verify(getRecordsRetrievalStrategy, atLeast(1)).getRecords(eq(MAX_RECORDS_PER_CALL)); } + @Test + public void testGetRecordsWithInvalidResponse() { + record = Record.builder().data(createByteBufferWithSize(SIZE_512_KB)).build(); + + when(records.size()).thenReturn(1000); + + GetRecordsResponse response = GetRecordsResponse.builder().records(records).build(); + when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(response); + when(dataFetcher.isShardEndReached()).thenReturn(false); + + getRecordsCache.start(sequenceNumber, initialPosition); + + try { + ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L) + .processRecordsInput(); + } catch (Exception e) { + assertEquals("No records found", e.getMessage()); + } + } + + @Test + public void testGetRecordsWithShardEnd() { + records = new ArrayList<>(); + + final List expectedRecords = new ArrayList<>(); + + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add("shardId"); + ChildShard leftChild = ChildShard.builder() + .shardId("shardId-000000000001") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("shardId-000000000002") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + + GetRecordsResponse response = GetRecordsResponse.builder().records(records).childShards(childShards).build(); + when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(response); + when(dataFetcher.isShardEndReached()).thenReturn(true); + + getRecordsCache.start(sequenceNumber, initialPosition); + ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L) + .processRecordsInput(); + + assertEquals(expectedRecords, result.records()); + assertEquals(childShards, result.childShards()); + assertTrue(result.isAtShardEnd()); + } + // TODO: Broken test @Test @Ignore @@ -269,7 +373,7 @@ public class PrefetchRecordsPublisherTest { @Test public void testRetryableRetrievalExceptionContinues() { - GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L).records(Collections.emptyList()).build(); + GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L).records(Collections.emptyList()).nextShardIterator(NEXT_SHARD_ITERATOR).build(); when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenThrow(new RetryableRetrievalException("Timeout", new TimeoutException("Timeout"))).thenReturn(response); getRecordsCache.start(sequenceNumber, initialPosition); @@ -292,7 +396,7 @@ public class PrefetchRecordsPublisherTest { when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenAnswer( i -> GetRecordsResponse.builder().records( Record.builder().data(SdkBytes.fromByteArray(new byte[] { 1, 2, 3 })).sequenceNumber(++sequenceNumberInResponse[0] + "").build()) - .build()); + .nextShardIterator(NEXT_SHARD_ITERATOR).build()); getRecordsCache.start(sequenceNumber, initialPosition); @@ -383,7 +487,7 @@ public class PrefetchRecordsPublisherTest { // to the subscriber. GetRecordsResponse response = GetRecordsResponse.builder().records( Record.builder().data(SdkBytes.fromByteArray(new byte[] { 1, 2, 3 })).sequenceNumber("123").build()) - .build(); + .nextShardIterator(NEXT_SHARD_ITERATOR).build(); when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenReturn(response); getRecordsCache.start(sequenceNumber, initialPosition); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java index 81ad5b6d..ddc25e21 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.retrieval.polling; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.when; import org.junit.Before; import org.junit.Ignore; @@ -23,6 +24,7 @@ import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.DataFetchingStrategy; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @@ -37,11 +39,15 @@ public class RecordsFetcherFactoryTest { private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @Mock private MetricsFactory metricsFactory; + @Mock + private DataFetcher dataFetcher; @Before public void setUp() { MockitoAnnotations.initMocks(this); recordsFetcherFactory = new SimpleRecordsFetcherFactory(); + when(getRecordsRetrievalStrategy.dataFetcher()).thenReturn(dataFetcher); + when(dataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("stream")); } @Test @@ -60,5 +66,4 @@ public class RecordsFetcherFactoryTest { metricsFactory, 1); assertThat(recordsCache, instanceOf(PrefetchRecordsPublisher.class)); } - } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java index fa10557f..0d68e51b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java @@ -35,4 +35,17 @@ public class BlockingUtils { throw new RuntimeException("No records found"); } } + + public static boolean blockUntilConditionSatisfied(Supplier conditionSupplier, long timeoutMillis) { + while(!conditionSupplier.get() && timeoutMillis > 0 ) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + timeoutMillis -= 100; + } + return conditionSupplier.get(); + } + } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java index a89ebef6..1aeddc60 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java @@ -45,6 +45,7 @@ public class ProcessRecordsInputMatcher extends TypeSafeDiagnosingMatcheramazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.2.12-SNAPSHOT + 2.3.0 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. @@ -33,7 +33,7 @@ - 2.13.25 + 2.14.0