diff --git a/CHANGELOG.md b/CHANGELOG.md
index 83dcfcce..3188791a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,5 +1,52 @@
# Changelog
-## Latest Release (1.13.3 March 2, 2020)
+
+## Latest Release (1.14.0 - August 17, 2020)
+
+* [Milestone#50](https://github.com/awslabs/amazon-kinesis-client/milestone/50)
+
+* 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 `KinesisClientLibConfiguration#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` API 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 KCL 1.x applications, this uses the `ChildShards` response of the `GetRecords` API.
+ * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.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. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.SHARD_END`, `PeriodicShardSyncManager#leasesRecoveryAuditorInconsistencyConfidenceThreshold` will be used to determine the threshold for number of consecutive scans containing holes in the lease table after which to enforce a shard sync. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.PERIODIC`, `leasesRecoveryAuditorInconsistencyConfidenceThreshold` is ignored.
+ * New configuration options are available to configure `PeriodicShardSyncManager` in `KinesisClientLibConfiguration`
+
+ | Name | Default | Description |
+ | ----------------------------------------------------- | ------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
+ | 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. Only used for `ShardSyncStrategyType.SHARD_END`. |
+
+ * 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. |
+
+* Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards.
+* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstShardPrioritization` while upgrading.
+* Upgrading version of AWS SDK to 1.11.844.
+* [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4.
+* [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy.
+
+## Release 1.13.3 (1.13.3 March 2, 2020)
[Milestone#49] (https://github.com/awslabs/amazon-kinesis-client/milestone/49)
* Refactoring shard closure verification performed by ShutdownTask.
* [PR #684] (https://github.com/awslabs/amazon-kinesis-client/pull/684)
diff --git a/README.md b/README.md
index 8db8b2d7..f936d602 100644
--- a/README.md
+++ b/README.md
@@ -31,28 +31,51 @@ To make it easier for developers to write record processors in other languages,
## Release Notes
-#### Latest Release (1.13.3 March 2, 2020)
-* Refactoring shard closure verification performed by ShutdownTask.
- * [PR #684] (https://github.com/awslabs/amazon-kinesis-client/pull/684)
-* Fixing the bug in ShardSyncTaskManager to resolve the issue of new shards not being processed after resharding.
- * [PR #694] (https://github.com/awslabs/amazon-kinesis-client/pull/694)
+### Latest Release (1.14.0 - August 17, 2020)
-#### Release (1.13.2 Janurary 13, 2020)
-* Adding backward compatible constructors that use the default DDB Billing Mode (#673)
- * [PR #673](https://github.com/awslabs/amazon-kinesis-client/pull/673)
+* [Milestone#50](https://github.com/awslabs/amazon-kinesis-client/milestone/50)
-#### Release (1.13.1 December 30, 2019)
-* Adding BillingMode Support to KCL 1.x. This enables the customer to specify if they want provisioned capacity for DDB, or pay per request.
- * [PR #656](https://github.com/awslabs/amazon-kinesis-client/pull/656)
-* Ensure ShardSyncTask invocation from ShardSyncTaskManager for pending ShardEnd events.
- * [PR #659](https://github.com/awslabs/amazon-kinesis-client/pull/659)
-* Fix the LeaseManagementIntegrationTest failure.
- * [PR #670](https://github.com/awslabs/amazon-kinesis-client/pull/670)
+* 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 `KinesisClientLibConfiguration#initialPositionInStreamExtended`.
+ * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html).
-#### Release (1.13.0 November 5, 2019)
-* Handling completed and blocked tasks better during graceful shutdown
- * [PR #640](https://github.com/awslabs/amazon-kinesis-client/pull/640)
+* Introducing support for the `ChildShards` response of the `GetRecords` API 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 KCL 1.x applications, this uses the `ChildShards` response of the `GetRecords` API.
+ * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.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. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.SHARD_END`, `PeriodicShardSyncManager#leasesRecoveryAuditorInconsistencyConfidenceThreshold` will be used to determine the threshold for number of consecutive scans containing holes in the lease table after which to enforce a shard sync. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.PERIODIC`, `leasesRecoveryAuditorInconsistencyConfidenceThreshold` is ignored.
+ * New configuration options are available to configure `PeriodicShardSyncManager` in `KinesisClientLibConfiguration`
+
+ | Name | Default | Description |
+ | ----------------------------------------------------- | ------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- |
+ | 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. Only used for `ShardSyncStrategyType.SHARD_END`. |
+
+ * 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. |
+
+* Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards.
+* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstShardPrioritization` while upgrading.
+* Upgrading version of AWS SDK to 1.11.844.
+* [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4.
+* [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy.
###### For remaining release notes check **[CHANGELOG.md][changelog-md]**.
diff --git a/pom.xml b/pom.xml
index 35f8f05c..48a802ae 100644
--- a/pom.xml
+++ b/pom.xml
@@ -6,7 +6,7 @@
amazon-kinesis-client
jar
Amazon Kinesis Client Library for Java
- 1.13.4-SNAPSHOT
+ 1.14.0
The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
from Amazon Kinesis.
@@ -25,7 +25,7 @@
- 1.11.728
+ 1.11.844
1.0.392
libsqlite4java
${project.build.directory}/test-lib
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java
index 47b1239f..5cf55dbf 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java
@@ -199,7 +199,7 @@ class ConsumerStates {
@Override
public ConsumerState shutdownTransition(ShutdownReason shutdownReason) {
- return ShardConsumerState.SHUTDOWN_COMPLETE.getConsumerState();
+ return ShardConsumerState.SHUTTING_DOWN.getConsumerState();
}
@Override
@@ -530,7 +530,9 @@ class ConsumerStates {
consumer.isIgnoreUnexpectedChildShards(),
consumer.getLeaseCoordinator(),
consumer.getTaskBackoffTimeMillis(),
- consumer.getGetRecordsCache(), consumer.getShardSyncer(), consumer.getShardSyncStrategy());
+ consumer.getGetRecordsCache(), consumer.getShardSyncer(),
+ consumer.getShardSyncStrategy(), consumer.getChildShards(),
+ consumer.getLeaseCleanupManager());
}
@Override
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java
new file mode 100644
index 00000000..3679f4dc
--- /dev/null
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java
@@ -0,0 +1,98 @@
+/*
+ * 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 com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
+import com.amazonaws.services.kinesis.leases.impl.Lease;
+import com.amazonaws.services.kinesis.model.Shard;
+import lombok.AllArgsConstructor;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisShardSyncer.StartingSequenceNumberAndShardIdBasedComparator;
+
+/**
+ * Class to help create leases when the table is initially empty.
+ */
+@AllArgsConstructor
+class EmptyLeaseTableSynchronizer implements LeaseSynchronizer {
+
+ private static final Log LOG = LogFactory.getLog(EmptyLeaseTableSynchronizer.class);
+
+ /**
+ * 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
+ * @return
+ */
+ @Override
+ public List determineNewLeasesToCreate(List shards,
+ List currentLeases,
+ InitialPositionInStreamExtended initialPosition,
+ Set inconsistentShardIds) {
+
+ final Map shardIdToShardMapOfAllKinesisShards =
+ KinesisShardSyncer.constructShardIdToShardMap(shards);
+
+ currentLeases.forEach(lease -> LOG.debug("Existing lease: " + lease.getLeaseKey()));
+
+ final List newLeasesToCreate =
+ getLeasesToCreateForOpenAndClosedShards(initialPosition, shards);
+
+ final Comparator startingSequenceNumberComparator =
+ new StartingSequenceNumberAndShardIdBasedComparator(shardIdToShardMapOfAllKinesisShards);
+
+ 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) {
+
+ final Map shardIdToNewLeaseMap = new HashMap<>();
+
+ for (Shard shard : shards) {
+ final String shardId = shard.getShardId();
+ final KinesisClientLease lease = KinesisShardSyncer.newKCLLease(shard);
+
+ final ExtendedSequenceNumber checkpoint = KinesisShardSyncer.convertToCheckpoint(initialPosition);
+ lease.setCheckpoint(checkpoint);
+
+ LOG.debug("Need to create a lease for shard with shardId " + shardId);
+ shardIdToNewLeaseMap.put(shardId, lease);
+ }
+
+ return new ArrayList(shardIdToNewLeaseMap.values());
+ }
+}
\ No newline at end of file
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java
index ba01bc9e..643b45d0 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java
@@ -14,6 +14,7 @@
*/
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+import java.time.Duration;
import java.util.Date;
import java.util.Optional;
import java.util.Set;
@@ -89,6 +90,23 @@ public class KinesisClientLibConfiguration {
*/
public static final boolean DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION = true;
+ /**
+ * Interval to run lease cleanup thread in {@link LeaseCleanupManager}.
+ */
+ private static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(1).toMillis();
+
+ /**
+ * Threshold in millis at which to check if there are any completed leases (leases for shards which have been
+ * closed as a result of a resharding operation) that need to be cleaned up.
+ */
+ private static final long DEFAULT_COMPLETED_LEASE_CLEANUP_THRESHOLD_MILLIS = Duration.ofMinutes(5).toMillis();
+
+ /**
+ * Threshold in millis at which to check if there are any garbage leases (leases for shards which no longer exist
+ * in the stream) that need to be cleaned up.
+ */
+ private static final long DEFAULT_GARBAGE_LEASE_CLEANUP_THRESHOLD_MILLIS = Duration.ofMinutes(30).toMillis();
+
/**
* Backoff time in milliseconds for Amazon Kinesis Client Library tasks (in the event of failures).
*/
@@ -129,7 +147,7 @@ public class KinesisClientLibConfiguration {
/**
* User agent set when Amazon Kinesis Client Library makes AWS requests.
*/
- public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.13.4-SNAPSHOT";
+ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.14.0";
/**
* KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls
@@ -175,6 +193,16 @@ public class KinesisClientLibConfiguration {
*/
public static final ShardSyncStrategyType DEFAULT_SHARD_SYNC_STRATEGY_TYPE = ShardSyncStrategyType.SHARD_END;
+ /**
+ * Default Lease Recovery Auditor execution period for SHARD_END ShardSyncStrategyType.
+ */
+ public static final long LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS = 2 * 60 * 1000L;
+
+ /**
+ * Default Lease Recovery Auditor inconsistency confidence threshold for running full shard sync for SHARD_END ShardSyncStrategyType.
+ */
+ public static final int LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD = 3;
+
/**
* Default Shard prioritization strategy.
*/
@@ -200,6 +228,11 @@ public class KinesisClientLibConfiguration {
*/
public static final int DEFAULT_MAX_LIST_SHARDS_RETRY_ATTEMPTS = 50;
+ /**
+ * The number of times the {@link Worker} will try to initialize before giving up.
+ */
+ public static final int DEFAULT_MAX_INITIALIZATION_ATTEMPTS = 20;
+
@Getter
private BillingMode billingMode;
private String applicationName;
@@ -241,6 +274,11 @@ public class KinesisClientLibConfiguration {
private ShardPrioritization shardPrioritization;
private long shutdownGraceMillis;
private ShardSyncStrategyType shardSyncStrategyType;
+ private long leaseCleanupIntervalMillis;
+ private long completedLeaseCleanupThresholdMillis;
+ private long garbageLeaseCleanupThresholdMillis;
+ private long leasesRecoveryAuditorExecutionFrequencyMillis;
+ private int leasesRecoveryAuditorInconsistencyConfidenceThreshold;
@Getter
private Optional timeoutInSeconds = Optional.empty();
@@ -266,6 +304,9 @@ public class KinesisClientLibConfiguration {
@Getter
private int maxListShardsRetryAttempts = DEFAULT_MAX_LIST_SHARDS_RETRY_ATTEMPTS;
+ @Getter
+ private int maxInitializationAttempts = DEFAULT_MAX_INITIALIZATION_ATTEMPTS;
+
/**
* Constructor.
*
@@ -276,6 +317,7 @@ public class KinesisClientLibConfiguration {
* @param credentialsProvider Provides credentials used to sign AWS requests
* @param workerId Used to distinguish different workers/processes of a Kinesis application
*/
+ @Deprecated
public KinesisClientLibConfiguration(String applicationName,
String streamName,
AWSCredentialsProvider credentialsProvider,
@@ -295,6 +337,7 @@ public class KinesisClientLibConfiguration {
* @param cloudWatchCredentialsProvider Provides credentials used to access CloudWatch
* @param workerId Used to distinguish different workers/processes of a Kinesis application
*/
+ @Deprecated
public KinesisClientLibConfiguration(String applicationName,
String streamName,
AWSCredentialsProvider kinesisCredentialsProvider,
@@ -365,6 +408,7 @@ public class KinesisClientLibConfiguration {
*/
// CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES
+ @Deprecated
public KinesisClientLibConfiguration(String applicationName,
String streamName,
String kinesisEndpoint,
@@ -436,6 +480,7 @@ public class KinesisClientLibConfiguration {
*/
// CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES
+ @Deprecated
public KinesisClientLibConfiguration(String applicationName,
String streamName,
String kinesisEndpoint,
@@ -462,54 +507,14 @@ public class KinesisClientLibConfiguration {
String regionName,
long shutdownGraceMillis,
BillingMode billingMode) {
- // Check following values are greater than zero
- checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis);
- checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis);
- checkIsValuePositive("ParentShardPollIntervalMillis", parentShardPollIntervalMillis);
- checkIsValuePositive("ShardSyncIntervalMillis", shardSyncIntervalMillis);
- checkIsValuePositive("MaxRecords", (long) maxRecords);
- checkIsValuePositive("TaskBackoffTimeMillis", taskBackoffTimeMillis);
- checkIsValuePositive("MetricsBufferTimeMills", metricsBufferTimeMillis);
- checkIsValuePositive("MetricsMaxQueueSize", (long) metricsMaxQueueSize);
- checkIsValuePositive("ShutdownGraceMillis", shutdownGraceMillis);
- this.applicationName = applicationName;
- this.tableName = applicationName;
- this.streamName = streamName;
- this.kinesisEndpoint = kinesisEndpoint;
- this.dynamoDBEndpoint = dynamoDBEndpoint;
- this.initialPositionInStream = initialPositionInStream;
- this.kinesisCredentialsProvider = kinesisCredentialsProvider;
- this.dynamoDBCredentialsProvider = dynamoDBCredentialsProvider;
- this.cloudWatchCredentialsProvider = cloudWatchCredentialsProvider;
- this.failoverTimeMillis = failoverTimeMillis;
- this.maxRecords = maxRecords;
- this.idleTimeBetweenReadsInMillis = idleTimeBetweenReadsInMillis;
- this.callProcessRecordsEvenForEmptyRecordList = callProcessRecordsEvenForEmptyRecordList;
- this.parentShardPollIntervalMillis = parentShardPollIntervalMillis;
- this.shardSyncIntervalMillis = shardSyncIntervalMillis;
- this.cleanupLeasesUponShardCompletion = cleanupTerminatedShardsBeforeExpiry;
- this.workerIdentifier = workerId;
- this.kinesisClientConfig = checkAndAppendKinesisClientLibUserAgent(kinesisClientConfig);
- this.dynamoDBClientConfig = checkAndAppendKinesisClientLibUserAgent(dynamoDBClientConfig);
- this.cloudWatchClientConfig = checkAndAppendKinesisClientLibUserAgent(cloudWatchClientConfig);
- this.taskBackoffTimeMillis = taskBackoffTimeMillis;
- this.metricsBufferTimeMillis = metricsBufferTimeMillis;
- this.metricsMaxQueueSize = metricsMaxQueueSize;
- this.metricsLevel = DEFAULT_METRICS_LEVEL;
- this.metricsEnabledDimensions = DEFAULT_METRICS_ENABLED_DIMENSIONS;
- this.validateSequenceNumberBeforeCheckpointing = validateSequenceNumberBeforeCheckpointing;
- this.regionName = regionName;
- this.maxLeasesForWorker = DEFAULT_MAX_LEASES_FOR_WORKER;
- this.maxLeasesToStealAtOneTime = DEFAULT_MAX_LEASES_TO_STEAL_AT_ONE_TIME;
- this.initialLeaseTableReadCapacity = DEFAULT_INITIAL_LEASE_TABLE_READ_CAPACITY;
- this.initialLeaseTableWriteCapacity = DEFAULT_INITIAL_LEASE_TABLE_WRITE_CAPACITY;
- this.initialPositionInStreamExtended =
- InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream);
- this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST;
- this.shardSyncStrategyType = DEFAULT_SHARD_SYNC_STRATEGY_TYPE;
- this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION;
- this.recordsFetcherFactory = new SimpleRecordsFetcherFactory();
- this.billingMode = billingMode;
+
+ this(applicationName, streamName, kinesisEndpoint, dynamoDBEndpoint, initialPositionInStream, kinesisCredentialsProvider,
+ dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId, maxRecords, idleTimeBetweenReadsInMillis,
+ callProcessRecordsEvenForEmptyRecordList, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry,
+ kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig, taskBackoffTimeMillis, metricsBufferTimeMillis,
+ metricsMaxQueueSize, validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis, billingMode,
+ new SimpleRecordsFetcherFactory(), DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS, DEFAULT_COMPLETED_LEASE_CLEANUP_THRESHOLD_MILLIS,
+ DEFAULT_GARBAGE_LEASE_CLEANUP_THRESHOLD_MILLIS);
}
/**
@@ -548,6 +553,7 @@ public class KinesisClientLibConfiguration {
*/
// CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES
+ @Deprecated
public KinesisClientLibConfiguration(String applicationName,
String streamName,
String kinesisEndpoint,
@@ -573,6 +579,91 @@ public class KinesisClientLibConfiguration {
boolean validateSequenceNumberBeforeCheckpointing,
String regionName,
RecordsFetcherFactory recordsFetcherFactory) {
+
+
+ this(applicationName, streamName, kinesisEndpoint, dynamoDBEndpoint, initialPositionInStream, kinesisCredentialsProvider,
+ dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId, maxRecords, idleTimeBetweenReadsInMillis,
+ callProcessRecordsEvenForEmptyRecordList, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry,
+ kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig, taskBackoffTimeMillis, metricsBufferTimeMillis,
+ metricsMaxQueueSize, validateSequenceNumberBeforeCheckpointing, regionName, 0, DEFAULT_DDB_BILLING_MODE,
+ recordsFetcherFactory, DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS, DEFAULT_COMPLETED_LEASE_CLEANUP_THRESHOLD_MILLIS,
+ DEFAULT_GARBAGE_LEASE_CLEANUP_THRESHOLD_MILLIS);
+ }
+
+ /**
+ * @param applicationName Name of the Kinesis application
+ * By default the application name is included in the user agent string used to make AWS requests. This
+ * can assist with troubleshooting (e.g. distinguish requests made by separate applications).
+ * @param streamName Name of the Kinesis stream
+ * @param kinesisEndpoint Kinesis endpoint
+ * @param dynamoDBEndpoint DynamoDB endpoint
+ * @param initialPositionInStream One of LATEST or TRIM_HORIZON. The KinesisClientLibrary will start fetching
+ * records from that location in the stream when an application starts up for the first time and there
+ * are no checkpoints. If there are checkpoints, then we start from the checkpoint position.
+ * @param kinesisCredentialsProvider Provides credentials used to access Kinesis
+ * @param dynamoDBCredentialsProvider Provides credentials used to access DynamoDB
+ * @param cloudWatchCredentialsProvider Provides credentials used to access CloudWatch
+ * @param failoverTimeMillis Lease duration (leases not renewed within this period will be claimed by others)
+ * @param workerId Used to distinguish different workers/processes of a Kinesis application
+ * @param maxRecords Max records to read per Kinesis getRecords() call
+ * @param idleTimeBetweenReadsInMillis Idle time between calls to fetch data from Kinesis
+ * @param callProcessRecordsEvenForEmptyRecordList Call the IRecordProcessor::processRecords() API even if
+ * GetRecords returned an empty record list.
+ * @param parentShardPollIntervalMillis Wait for this long between polls to check if parent shards are done
+ * @param shardSyncIntervalMillis Time between tasks to sync leases and Kinesis shards
+ * @param cleanupTerminatedShardsBeforeExpiry Clean up shards we've finished processing (don't wait for expiration
+ * in Kinesis)
+ * @param kinesisClientConfig Client Configuration used by Kinesis client
+ * @param dynamoDBClientConfig Client Configuration used by DynamoDB client
+ * @param cloudWatchClientConfig Client Configuration used by CloudWatch client
+ * @param taskBackoffTimeMillis Backoff period when tasks encounter an exception
+ * @param metricsBufferTimeMillis Metrics are buffered for at most this long before publishing to CloudWatch
+ * @param metricsMaxQueueSize Max number of metrics to buffer before publishing to CloudWatch
+ * @param validateSequenceNumberBeforeCheckpointing whether KCL should validate client provided sequence numbers
+ * with a call to Amazon Kinesis before checkpointing for calls to
+ * {@link RecordProcessorCheckpointer#checkpoint(String)}
+ * @param regionName The region name for the service
+ * @param shutdownGraceMillis Time before gracefully shutdown forcefully terminates
+ * @param billingMode The DDB Billing mode to set for lease table creation.
+ * @param recordsFetcherFactory Factory to create the records fetcher to retrieve data from Kinesis for a given shard.
+ * @param leaseCleanupIntervalMillis Rate at which to run lease cleanup thread in
+ * {@link com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager}
+ * @param completedLeaseCleanupThresholdMillis Threshold in millis at which to check if there are any completed leases
+ * (leases for shards which have been closed as a result of a resharding operation) that need to be cleaned up.
+ * @param garbageLeaseCleanupThresholdMillis Threshold in millis at which to check if there are any garbage leases
+ * (leases for shards which no longer exist in the stream) that need to be cleaned up.
+ */
+ public KinesisClientLibConfiguration(String applicationName,
+ String streamName,
+ String kinesisEndpoint,
+ String dynamoDBEndpoint,
+ InitialPositionInStream initialPositionInStream,
+ AWSCredentialsProvider kinesisCredentialsProvider,
+ AWSCredentialsProvider dynamoDBCredentialsProvider,
+ AWSCredentialsProvider cloudWatchCredentialsProvider,
+ long failoverTimeMillis,
+ String workerId,
+ int maxRecords,
+ long idleTimeBetweenReadsInMillis,
+ boolean callProcessRecordsEvenForEmptyRecordList,
+ long parentShardPollIntervalMillis,
+ long shardSyncIntervalMillis,
+ boolean cleanupTerminatedShardsBeforeExpiry,
+ ClientConfiguration kinesisClientConfig,
+ ClientConfiguration dynamoDBClientConfig,
+ ClientConfiguration cloudWatchClientConfig,
+ long taskBackoffTimeMillis,
+ long metricsBufferTimeMillis,
+ int metricsMaxQueueSize,
+ boolean validateSequenceNumberBeforeCheckpointing,
+ String regionName,
+ long shutdownGraceMillis,
+ BillingMode billingMode,
+ RecordsFetcherFactory recordsFetcherFactory,
+ long leaseCleanupIntervalMillis,
+ long completedLeaseCleanupThresholdMillis,
+ long garbageLeaseCleanupThresholdMillis) {
+
// Check following values are greater than zero
checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis);
checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis);
@@ -617,9 +708,15 @@ public class KinesisClientLibConfiguration {
InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream);
this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST;
this.shardSyncStrategyType = DEFAULT_SHARD_SYNC_STRATEGY_TYPE;
+ this.leasesRecoveryAuditorExecutionFrequencyMillis = LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS;
+ this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD;
this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION;
this.recordsFetcherFactory = recordsFetcherFactory;
+ this.leaseCleanupIntervalMillis = leaseCleanupIntervalMillis;
+ this.completedLeaseCleanupThresholdMillis = completedLeaseCleanupThresholdMillis;
+ this.garbageLeaseCleanupThresholdMillis = garbageLeaseCleanupThresholdMillis;
this.shutdownGraceMillis = shutdownGraceMillis;
+ this.billingMode = billingMode;
}
// Check if value is positive, otherwise throw an exception
@@ -828,6 +925,29 @@ public class KinesisClientLibConfiguration {
return cleanupLeasesUponShardCompletion;
}
+ /**
+ * @return Interval in millis at which to run lease cleanup thread in {@link com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager}
+ */
+ public long leaseCleanupIntervalMillis() {
+ return leaseCleanupIntervalMillis;
+ }
+
+ /**
+ * @return Interval in millis at which to check if there are any completed leases (leases for shards which have been
+ * closed as a result of a resharding operation) that need to be cleaned up.
+ */
+ public long completedLeaseCleanupThresholdMillis() {
+ return completedLeaseCleanupThresholdMillis;
+ }
+
+ /**
+ * @return Interval in millis at which to check if there are any garbage leases (leases for shards which no longer
+ * exist in the stream) that need to be cleaned up.
+ */
+ public long garbageLeaseCleanupThresholdMillis() {
+ return garbageLeaseCleanupThresholdMillis;
+ }
+
/**
* @return true if we should ignore child shards which have open parents
*/
@@ -864,6 +984,20 @@ public class KinesisClientLibConfiguration {
return shardSyncStrategyType;
}
+ /**
+ * @return leasesRecoveryAuditorExecutionFrequencyMillis to be used by SHARD_END ShardSyncStrategyType.
+ */
+ public long getLeasesRecoveryAuditorExecutionFrequencyMillis() {
+ return leasesRecoveryAuditorExecutionFrequencyMillis;
+ }
+
+ /**
+ * @return leasesRecoveryAuditorInconsistencyConfidenceThreshold to be used by SHARD_END ShardSyncStrategyType.
+ */
+ public int getLeasesRecoveryAuditorInconsistencyConfidenceThreshold() {
+ return leasesRecoveryAuditorInconsistencyConfidenceThreshold;
+ }
+
/**
* @return Max leases this Worker can handle at a time
*/
@@ -1241,6 +1375,24 @@ public class KinesisClientLibConfiguration {
return this;
}
+ /**
+ * @param leasesRecoveryAuditorExecutionFrequencyMillis Leases Recovery Auditor Execution period.
+ * @return {@link KinesisClientLibConfiguration}
+ */
+ public KinesisClientLibConfiguration withLeasesRecoveryAuditorExecutionFrequencyMillis(long leasesRecoveryAuditorExecutionFrequencyMillis) {
+ this.leasesRecoveryAuditorExecutionFrequencyMillis = leasesRecoveryAuditorExecutionFrequencyMillis;
+ return this;
+ }
+
+ /**
+ * @param leasesRecoveryAuditorInconsistencyConfidenceThreshold Leases Recovery Auditor Execution inconsistency confidence threshold.
+ * @return {@link KinesisClientLibConfiguration}
+ */
+ public KinesisClientLibConfiguration withLeasesRecoveryAuditorInconsistencyConfidenceThreshold(int leasesRecoveryAuditorInconsistencyConfidenceThreshold) {
+ this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold;
+ return this;
+ }
+
/**
*
* @param regionName The region name for the service
@@ -1458,4 +1610,49 @@ public class KinesisClientLibConfiguration {
this.maxListShardsRetryAttempts = maxListShardsRetryAttempts;
return this;
}
+
+ /**
+ * @param maxInitializationAttempts Max number of Worker initialization attempts before giving up
+ * @return
+ */
+ public KinesisClientLibConfiguration withMaxInitializationAttempts(int maxInitializationAttempts) {
+ checkIsValuePositive("maxInitializationAttempts", maxInitializationAttempts);
+ this.maxInitializationAttempts = maxInitializationAttempts;
+ return this;
+ }
+
+ /**
+ * @param leaseCleanupIntervalMillis Rate at which to run lease cleanup thread in
+ * {@link com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager}
+ * @return
+ */
+ public KinesisClientLibConfiguration withLeaseCleanupIntervalMillis(long leaseCleanupIntervalMillis) {
+ checkIsValuePositive("leaseCleanupIntervalMillis", leaseCleanupIntervalMillis);
+ this.leaseCleanupIntervalMillis = leaseCleanupIntervalMillis;
+ return this;
+ }
+
+ /**
+ * Threshold in millis at which to check if there are any completed leases (leases for shards which have been
+ * closed as a result of a resharding operation) that need to be cleaned up.
+ * @param completedLeaseCleanupThresholdMillis
+ * @return
+ */
+ public KinesisClientLibConfiguration withCompletedLeaseCleanupThresholdMillis(long completedLeaseCleanupThresholdMillis) {
+ checkIsValuePositive("completedLeaseCleanupThresholdMillis", completedLeaseCleanupThresholdMillis);
+ this.completedLeaseCleanupThresholdMillis = completedLeaseCleanupThresholdMillis;
+ return this;
+ }
+
+ /**
+ * Threshold in millis at which to check if there are any garbage leases (leases for shards which no longer exist
+ * in the stream) that need to be cleaned up.
+ * @param garbageLeaseCleanupThresholdMillis
+ * @return
+ */
+ public KinesisClientLibConfiguration withGarbageLeaseCleanupThresholdMillis(long garbageLeaseCleanupThresholdMillis) {
+ checkIsValuePositive("garbageLeaseCleanupThresholdMillis", garbageLeaseCleanupThresholdMillis);
+ this.garbageLeaseCleanupThresholdMillis = garbageLeaseCleanupThresholdMillis;
+ return this;
+ }
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java
index e425e070..ae4e321d 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java
@@ -16,7 +16,10 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import java.util.Collections;
import java.util.Date;
+import java.util.List;
+import com.amazonaws.SdkClientException;
+import com.amazonaws.services.kinesis.model.ChildShard;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -47,6 +50,7 @@ class KinesisDataFetcher {
private boolean isInitialized;
private String lastKnownSequenceNumber;
private InitialPositionInStreamExtended initialPositionInStream;
+ private List childShards = Collections.emptyList();
/**
*
@@ -85,8 +89,11 @@ class KinesisDataFetcher {
final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() {
@Override
public GetRecordsResult getResult() {
- return new GetRecordsResult().withMillisBehindLatest(null).withRecords(Collections.emptyList())
- .withNextShardIterator(null);
+ return new GetRecordsResult()
+ .withMillisBehindLatest(null)
+ .withRecords(Collections.emptyList())
+ .withNextShardIterator(null)
+ .withChildShards(Collections.emptyList());
}
@Override
@@ -113,12 +120,20 @@ class KinesisDataFetcher {
@Override
public GetRecordsResult accept() {
+ if (!isValidResult(result)) {
+ // Throwing SDK exception when the GetRecords result is not valid. This will allow PrefetchGetRecordsCache to retry the GetRecords call.
+ throw new SdkClientException("Shard " + shardId +": GetRecordsResult is not valid. NextShardIterator: " + result.getNextShardIterator()
+ + ". ChildShards: " + result.getChildShards());
+ }
nextIterator = result.getNextShardIterator();
if (!CollectionUtils.isNullOrEmpty(result.getRecords())) {
lastKnownSequenceNumber = Iterables.getLast(result.getRecords()).getSequenceNumber();
}
if (nextIterator == null) {
- LOG.info("Reached shard end: nextIterator is null in AdvancingResult.accept for shard " + shardId);
+ LOG.info("Reached shard end: nextIterator is null in AdvancingResult.accept for shard " + shardId + ". childShards: " + result.getChildShards());
+ if (!CollectionUtils.isNullOrEmpty(result.getChildShards())) {
+ childShards = result.getChildShards();
+ }
isShardEndReached = true;
}
return getResult();
@@ -130,6 +145,23 @@ class KinesisDataFetcher {
}
}
+ private boolean isValidResult(GetRecordsResult getRecordsResult) {
+ // GetRecords result should contain childShard information. There are two valid combination for the nextShardIterator and childShards
+ // If the GetRecords call does not reach the shard end, getRecords result should contain a non-null nextShardIterator and an empty list of childShards.
+ // If the GetRecords call reaches the shard end, getRecords result should contain a null nextShardIterator and a non-empty list of childShards.
+ // All other combinations are invalid and indicating an issue with GetRecords result from Kinesis service.
+ if (getRecordsResult.getNextShardIterator() == null && CollectionUtils.isNullOrEmpty(getRecordsResult.getChildShards()) ||
+ getRecordsResult.getNextShardIterator() != null && !CollectionUtils.isNullOrEmpty(getRecordsResult.getChildShards())) {
+ return false;
+ }
+ for (ChildShard childShard : getRecordsResult.getChildShards()) {
+ if (CollectionUtils.isNullOrEmpty(childShard.getParentShards())) {
+ return false;
+ }
+ }
+ return true;
+ }
+
/**
* Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number.
* @param initialCheckpoint Current checkpoint sequence number for this shard.
@@ -141,8 +173,7 @@ class KinesisDataFetcher {
isInitialized = true;
}
- public void initialize(ExtendedSequenceNumber initialCheckpoint,
- InitialPositionInStreamExtended initialPositionInStream) {
+ public void initialize(ExtendedSequenceNumber initialCheckpoint, InitialPositionInStreamExtended initialPositionInStream) {
LOG.info("Initializing shard " + shardId + " with " + initialCheckpoint.getSequenceNumber());
advanceIteratorTo(initialCheckpoint.getSequenceNumber(), initialPositionInStream);
isInitialized = true;
@@ -171,6 +202,7 @@ class KinesisDataFetcher {
if (nextIterator == null) {
LOG.info("Reached shard end: cannot advance iterator for shard " + shardId);
isShardEndReached = true;
+ // TODO: transition to ShuttingDown state on shardend instead to shutdown state for enqueueing this for cleanup
}
this.lastKnownSequenceNumber = sequenceNumber;
this.initialPositionInStream = initialPositionInStream;
@@ -248,6 +280,10 @@ class KinesisDataFetcher {
return isShardEndReached;
}
+ protected List getChildShards() {
+ return childShards;
+ }
+
/** Note: This method has package level access for testing purposes.
* @return nextIterator
*/
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java
index 31d8d998..514cfb8c 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java
@@ -10,6 +10,7 @@ import java.util.Set;
/**
* Represents the class that decides if a lease is eligible for cleanup.
*/
+@Deprecated
class KinesisLeaseCleanupValidator implements LeaseCleanupValidator {
private static final Log LOG = LogFactory.getLog(KinesisLeaseCleanupValidator.class);
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java
index c23fd678..4235bad9 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java
@@ -17,8 +17,6 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import java.io.Serializable;
import java.math.BigInteger;
import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
@@ -26,7 +24,11 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import com.amazonaws.services.kinesis.model.ChildShard;
+import com.amazonaws.services.kinesis.model.ShardFilter;
+import com.amazonaws.services.kinesis.model.ShardFilterType;
import com.amazonaws.util.CollectionUtils;
+import lombok.NoArgsConstructor;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.lang3.StringUtils;
@@ -59,11 +61,10 @@ class KinesisShardSyncer implements ShardSyncer {
}
synchronized void bootstrapShardLeases(IKinesisProxy kinesisProxy, ILeaseManager leaseManager,
- InitialPositionInStreamExtended initialPositionInStream, boolean cleanupLeasesOfCompletedShards,
- boolean ignoreUnexpectedChildShards)
- throws DependencyException, InvalidStateException, ProvisionedThroughputException,
- KinesisClientLibIOException {
- syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards,
+ InitialPositionInStreamExtended initialPositionInStream,
+ boolean ignoreUnexpectedChildShards)
+ throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
+ syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream,
ignoreUnexpectedChildShards);
}
@@ -86,7 +87,7 @@ class KinesisShardSyncer implements ShardSyncer {
boolean ignoreUnexpectedChildShards)
throws DependencyException, InvalidStateException, ProvisionedThroughputException,
KinesisClientLibIOException {
- syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards);
+ syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, ignoreUnexpectedChildShards);
}
/**
@@ -109,7 +110,8 @@ class KinesisShardSyncer implements ShardSyncer {
boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards, List latestShards)
throws DependencyException, InvalidStateException, ProvisionedThroughputException,
KinesisClientLibIOException {
- syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, latestShards);
+ syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream,
+ ignoreUnexpectedChildShards, latestShards, leaseManager.isLeaseTableEmpty());
}
/**
@@ -118,7 +120,6 @@ class KinesisShardSyncer implements ShardSyncer {
* @param kinesisProxy
* @param leaseManager
* @param initialPosition
- * @param cleanupLeasesOfCompletedShards
* @param ignoreUnexpectedChildShards
* @throws DependencyException
* @throws InvalidStateException
@@ -126,12 +127,21 @@ class KinesisShardSyncer implements ShardSyncer {
* @throws KinesisClientLibIOException
*/
private synchronized void syncShardLeases(IKinesisProxy kinesisProxy,
- ILeaseManager leaseManager, InitialPositionInStreamExtended initialPosition,
- boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards)
- throws DependencyException, InvalidStateException, ProvisionedThroughputException,
- KinesisClientLibIOException {
- List latestShards = getShardList(kinesisProxy);
- syncShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, latestShards);
+ ILeaseManager leaseManager,
+ InitialPositionInStreamExtended initialPosition,
+ boolean ignoreUnexpectedChildShards)
+ throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
+
+ // In the case where the lease table is empty, we want to synchronize the minimal amount of shards possible
+ // based on the given initial position.
+ // TODO: Implement shard list filtering on non-empty lease table case
+ final boolean isLeaseTableEmpty = leaseManager.isLeaseTableEmpty();
+ final List latestShards = isLeaseTableEmpty
+ ? getShardListAtInitialPosition(kinesisProxy, initialPosition)
+ : getCompleteShardList(kinesisProxy);
+
+ syncShardLeases(kinesisProxy, leaseManager, initialPosition,
+ ignoreUnexpectedChildShards, latestShards, isLeaseTableEmpty);
}
/**
@@ -140,7 +150,6 @@ class KinesisShardSyncer implements ShardSyncer {
* @param kinesisProxy
* @param leaseManager
* @param initialPosition
- * @param cleanupLeasesOfCompletedShards
* @param ignoreUnexpectedChildShards
* @param latestShards latest snapshot of shards to reuse
* @throws DependencyException
@@ -150,13 +159,17 @@ class KinesisShardSyncer implements ShardSyncer {
*/
// CHECKSTYLE:OFF CyclomaticComplexity
private synchronized void syncShardLeases(IKinesisProxy kinesisProxy,
- ILeaseManager leaseManager, InitialPositionInStreamExtended initialPosition,
- boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards, List latestShards)
+ ILeaseManager leaseManager,
+ InitialPositionInStreamExtended initialPosition,
+ boolean ignoreUnexpectedChildShards,
+ List latestShards,
+ boolean isLeaseTableEmpty)
throws DependencyException, InvalidStateException, ProvisionedThroughputException,
KinesisClientLibIOException {
+
List shards;
if(CollectionUtils.isNullOrEmpty(latestShards)) {
- shards = getShardList(kinesisProxy);
+ shards = isLeaseTableEmpty ? getShardListAtInitialPosition(kinesisProxy, initialPosition) : getCompleteShardList(kinesisProxy);
} else {
shards = latestShards;
}
@@ -169,11 +182,16 @@ class KinesisShardSyncer implements ShardSyncer {
assertAllParentShardsAreClosed(inconsistentShardIds);
}
- List currentLeases = leaseManager.listLeases();
+ // Determine which lease sync strategy to use based on the state of the lease table
+ final LeaseSynchronizer leaseSynchronizer = isLeaseTableEmpty
+ ? new EmptyLeaseTableSynchronizer()
+ : new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap);
- List newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition,
- inconsistentShardIds);
+ final List currentLeases = leaseManager.listLeases();
+ final List newLeasesToCreate = determineNewLeasesToCreate(leaseSynchronizer, shards,
+ currentLeases, initialPosition, inconsistentShardIds);
LOG.debug("Num new leases to create: " + newLeasesToCreate.size());
+
for (KinesisClientLease lease : newLeasesToCreate) {
long startTimeMillis = System.currentTimeMillis();
boolean success = false;
@@ -190,11 +208,6 @@ class KinesisShardSyncer implements ShardSyncer {
trackedLeases.addAll(currentLeases);
}
trackedLeases.addAll(newLeasesToCreate);
- cleanupGarbageLeases(shards, trackedLeases, kinesisProxy, leaseManager);
- if (cleanupLeasesOfCompletedShards) {
- cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases,
- leaseManager);
- }
}
// CHECKSTYLE:ON CyclomaticComplexity
@@ -317,7 +330,7 @@ class KinesisShardSyncer implements ShardSyncer {
* @param shardIdToShardMap
* @return
*/
- Map> constructShardIdToChildShardIdsMap(Map shardIdToShardMap) {
+ static Map> constructShardIdToChildShardIdsMap(Map shardIdToShardMap) {
Map> shardIdToChildShardIdsMap = new HashMap<>();
for (Map.Entry entry : shardIdToShardMap.entrySet()) {
String shardId = entry.getKey();
@@ -345,7 +358,7 @@ class KinesisShardSyncer implements ShardSyncer {
return shardIdToChildShardIdsMap;
}
- private List getShardList(IKinesisProxy kinesisProxy) throws KinesisClientLibIOException {
+ private List getCompleteShardList(IKinesisProxy kinesisProxy) throws KinesisClientLibIOException {
List shards = kinesisProxy.getShardList();
if (shards == null) {
throw new KinesisClientLibIOException(
@@ -354,46 +367,50 @@ class KinesisShardSyncer implements ShardSyncer {
return shards;
}
+ private List getShardListAtInitialPosition(IKinesisProxy kinesisProxy,
+ InitialPositionInStreamExtended initialPosition)
+ throws KinesisClientLibIOException {
+
+ final ShardFilter shardFilter = getShardFilterAtInitialPosition(initialPosition);
+ final List shards = kinesisProxy.getShardListWithFilter(shardFilter);
+
+ if (shards == null) {
+ throw new KinesisClientLibIOException(
+ "Stream is not in ACTIVE OR UPDATING state - will retry getting the shard list.");
+ }
+
+ return shards;
+ }
+
+ private static ShardFilter getShardFilterAtInitialPosition(InitialPositionInStreamExtended initialPosition) {
+ ShardFilter shardFilter = new ShardFilter();
+
+ switch (initialPosition.getInitialPositionInStream()) {
+ case LATEST:
+ shardFilter = shardFilter.withType(ShardFilterType.AT_LATEST);
+ break;
+ case TRIM_HORIZON:
+ shardFilter = shardFilter.withType(ShardFilterType.AT_TRIM_HORIZON);
+ break;
+ case AT_TIMESTAMP:
+ shardFilter = shardFilter.withType(ShardFilterType.AT_TIMESTAMP)
+ .withTimestamp(initialPosition.getTimestamp());
+ break;
+ default:
+ throw new IllegalArgumentException(initialPosition.getInitialPositionInStream()
+ + " is not a supported initial position in a Kinesis stream. Supported initial positions are"
+ + " AT_LATEST, AT_TRIM_HORIZON, and AT_TIMESTAMP.");
+ }
+
+ return shardFilter;
+ }
+
/**
* 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.
- *
- * 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 to use when updating leases based on the current state of
+ * the lease table (empty vs. non-empty)
* @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
@@ -401,91 +418,33 @@ class KinesisShardSyncer implements ShardSyncer {
* @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
*/
- List determineNewLeasesToCreate(List shards, List currentLeases,
- InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds) {
- Map shardIdToNewLeaseMap = new HashMap();
- Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards);
+ List determineNewLeasesToCreate(LeaseSynchronizer leaseSynchronizer,
+ List shards,
+ List currentLeases,
+ InitialPositionInStreamExtended initialPosition,
+ Set inconsistentShardIds) {
- Set shardIdsOfCurrentLeases = new HashSet();
- for (KinesisClientLease lease : currentLeases) {
- shardIdsOfCurrentLeases.add(lease.getLeaseKey());
- LOG.debug("Existing lease: " + lease);
- }
-
- List openShards = getOpenShards(shards);
- Map memoizationContext = new HashMap<>();
-
- // Iterate over the open shards and find those that don't have any lease entries.
- for (Shard shard : openShards) {
- String shardId = shard.getShardId();
- LOG.debug("Evaluating leases for open shard " + shardId + " and its ancestors.");
- if (shardIdsOfCurrentLeases.contains(shardId)) {
- LOG.debug("Lease for shardId " + shardId + " already exists. Not creating a lease");
- } else if (inconsistentShardIds.contains(shardId)) {
- LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease");
- } else {
- LOG.debug("Need to create a lease for shardId " + shardId);
- KinesisClientLease newLease = newKCLLease(shard);
- 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.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
- } else {
- newLease.setCheckpoint(convertToCheckpoint(initialPosition));
- }
- LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint());
- shardIdToNewLeaseMap.put(shardId, newLease);
- }
- }
-
- List newLeasesToCreate = new ArrayList();
- newLeasesToCreate.addAll(shardIdToNewLeaseMap.values());
- Comparator super KinesisClientLease> startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator(
- shardIdToShardMapOfAllKinesisShards);
- Collections.sort(newLeasesToCreate, startingSequenceNumberComparator);
- return newLeasesToCreate;
+ return leaseSynchronizer.determineNewLeasesToCreate(shards, currentLeases, initialPosition,
+ inconsistentShardIds);
}
/**
* Determine new leases to create and their initial checkpoint.
* Note: Package level access only for testing purposes.
*/
- List determineNewLeasesToCreate(List shards, List currentLeases,
- InitialPositionInStreamExtended initialPosition) {
+ List determineNewLeasesToCreate(LeaseSynchronizer leaseSynchronizer,
+ List shards,
+ List currentLeases,
+ InitialPositionInStreamExtended initialPosition) {
+
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.
@@ -498,12 +457,13 @@ class KinesisShardSyncer implements ShardSyncer {
* @return true if the shard is a descendant of any current shard (lease already exists)
*/
// CHECKSTYLE:OFF CyclomaticComplexity
- boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId,
+ static boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId,
InitialPositionInStreamExtended initialPosition, Set shardIdsOfCurrentLeases,
Map shardIdToShardMapOfAllKinesisShards,
- Map shardIdToLeaseMapOfNewShards, Map memoizationContext) {
+ Map shardIdToLeaseMapOfNewShards, MemoizationContext memoizationContext) {
+
+ final Boolean previousValue = memoizationContext.isDescendant(shardId);
- Boolean previousValue = memoizationContext.get(shardId);
if (previousValue != null) {
return previousValue;
}
@@ -523,10 +483,13 @@ class KinesisShardSyncer implements ShardSyncer {
shard = shardIdToShardMapOfAllKinesisShards.get(shardId);
parentShardIds = getParentShardIds(shard, shardIdToShardMapOfAllKinesisShards);
for (String parentShardId : parentShardIds) {
- // Check if the parent is a descendant, and include its ancestors.
- if (checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, initialPosition,
- shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards,
- memoizationContext)) {
+ // Check if the parent is a descendant, and include its ancestors. Or, if the parent is NOT a
+ // descendant but we should create a lease for it anyway (e.g. to include in processing from
+ // TRIM_HORIZON or AT_TIMESTAMP). If either is true, then we mark the current shard as a descendant.
+ final boolean isParentDescendant = checkIfDescendantAndAddNewLeasesForAncestors(parentShardId,
+ initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards,
+ shardIdToLeaseMapOfNewShards, memoizationContext);
+ if (isParentDescendant || memoizationContext.shouldCreateLease(parentShardId)) {
isDescendant = true;
descendantParentShardIds.add(parentShardId);
LOG.debug("Parent shard " + parentShardId + " is a descendant.");
@@ -539,37 +502,76 @@ class KinesisShardSyncer implements ShardSyncer {
if (isDescendant) {
for (String parentShardId : parentShardIds) {
if (!shardIdsOfCurrentLeases.contains(parentShardId)) {
- LOG.debug("Need to create a lease for shardId " + parentShardId);
KinesisClientLease lease = shardIdToLeaseMapOfNewShards.get(parentShardId);
+
+ // If the lease for the parent shard does not already exist, there are two cases in which we
+ // would want to create it:
+ // - If we have already marked the parentShardId for lease creation in a prior recursive
+ // call. This could happen if we are trying to process from TRIM_HORIZON or AT_TIMESTAMP.
+ // - If the parent shard is not a descendant but the current shard is a descendant, then
+ // the parent shard is the oldest shard in the shard hierarchy that does not have an
+ // ancestor in the lease table (the adjacent parent is necessarily a descendant, and
+ // therefore covered in the lease table). So we should create a lease for the parent.
+
if (lease == null) {
- lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId));
- shardIdToLeaseMapOfNewShards.put(parentShardId, lease);
+ if (memoizationContext.shouldCreateLease(parentShardId) ||
+ !descendantParentShardIds.contains(parentShardId)) {
+ LOG.debug("Need to create a lease for shardId " + parentShardId);
+ lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId));
+ shardIdToLeaseMapOfNewShards.put(parentShardId, lease);
+ }
}
- if (descendantParentShardIds.contains(parentShardId) && !initialPosition
- .getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) {
- lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
- } else {
- lease.setCheckpoint(convertToCheckpoint(initialPosition));
+ /**
+ * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the
+ * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will
+ * add a lease just like we do for TRIM_HORIZON. However we will only return back records
+ * with server-side timestamp at or after the specified initial position timestamp.
+ *
+ * Shard structure (each level depicts a stream segment):
+ * 0 1 2 3 4 5 - shards till epoch 102
+ * \ / \ / | |
+ * 6 7 4 5 - shards from epoch 103 - 205
+ * \ / | /\
+ * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber)
+ *
+ * Current leases: (4, 5, 7)
+ *
+ * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with
+ * timestamp value 206. We will then create new leases for all the shards 0 and 1 (with
+ * checkpoint set AT_TIMESTAMP), even though these ancestor shards have an epoch less than
+ * 206. However as we begin processing the ancestor shards, their checkpoints would be
+ * updated to SHARD_END and their leases would then be deleted since they won't have records
+ * with server-side timestamp at/after 206. And after that we will begin processing the
+ * descendant shards with epoch at/after 206 and we will return the records that meet the
+ * timestamp requirement for these shards.
+ */
+ if (lease != null) {
+ if (descendantParentShardIds.contains(parentShardId) && !initialPosition
+ .getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) {
+ lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
+ } else {
+ lease.setCheckpoint(convertToCheckpoint(initialPosition));
+ }
}
}
}
} else {
- // This shard should be included, if the customer wants to process all records in the stream or
- // if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a lease just like we do
- // for TRIM_HORIZON. However we will only return back records with server-side timestamp at or
- // after the specified initial position timestamp.
+ // This shard is not a descendant, but should still be included if the customer wants to process all
+ // records in the stream or if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a
+ // lease just like we do for TRIM_HORIZON. However we will only return back records with server-side
+ // timestamp at or after the specified initial position timestamp.
if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)
|| initialPosition.getInitialPositionInStream()
.equals(InitialPositionInStream.AT_TIMESTAMP)) {
- isDescendant = true;
+ memoizationContext.setShouldCreateLease(shardId, true);
}
}
}
}
- memoizationContext.put(shardId, isDescendant);
+ memoizationContext.setIsDescendant(shardId, isDescendant);
return isDescendant;
}
// CHECKSTYLE:ON CyclomaticComplexity
@@ -583,7 +585,7 @@ class KinesisShardSyncer implements ShardSyncer {
* @param shardIdToShardMapOfAllKinesisShards ShardId->Shard map containing all shards obtained via DescribeStream.
* @return Set of parentShardIds
*/
- Set getParentShardIds(Shard shard, Map shardIdToShardMapOfAllKinesisShards) {
+ static Set getParentShardIds(Shard shard, Map shardIdToShardMapOfAllKinesisShards) {
Set parentShardIds = new HashSet(2);
String parentShardId = shard.getParentShardId();
if ((parentShardId != null) && shardIdToShardMapOfAllKinesisShards.containsKey(parentShardId)) {
@@ -596,150 +598,6 @@ class KinesisShardSyncer implements ShardSyncer {
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, AND
- * * 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 kinesisProxy Kinesis proxy (used to get shard list)
- * @param leaseManager
- * @throws KinesisClientLibIOException Thrown if we couldn't get a fresh shard list from Kinesis.
- * @throws ProvisionedThroughputException
- * @throws InvalidStateException
- * @throws DependencyException
- */
- private void cleanupGarbageLeases(List shards, List trackedLeases,
- IKinesisProxy kinesisProxy, ILeaseManager leaseManager)
- throws KinesisClientLibIOException, DependencyException, InvalidStateException,
- ProvisionedThroughputException {
- Set kinesisShards = new HashSet<>();
- for (Shard shard : shards) {
- kinesisShards.add(shard.getShardId());
- }
-
- // Check if there are leases for non-existent shards
- List garbageLeases = new ArrayList<>();
- for (KinesisClientLease lease : trackedLeases) {
- if (leaseCleanupValidator.isCandidateForCleanup(lease, kinesisShards)) {
- garbageLeases.add(lease);
- }
- }
-
- if (!garbageLeases.isEmpty()) {
- LOG.info("Found " + garbageLeases.size() + " candidate leases for cleanup. Refreshing list of"
- + " Kinesis shards to pick up recent/latest shards");
- List currentShardList = getShardList(kinesisProxy);
- Set currentKinesisShardIds = new HashSet<>();
- for (Shard shard : currentShardList) {
- currentKinesisShardIds.add(shard.getShardId());
- }
-
- for (KinesisClientLease lease : garbageLeases) {
- if (leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds)) {
- LOG.info("Deleting lease for shard " + lease.getLeaseKey()
- + " as it is not present in Kinesis stream.");
- leaseManager.deleteLease(lease);
- }
- }
- }
-
- }
-
- /**
- * 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 leaseManager Lease manager (will be used to delete leases)
- * @throws DependencyException
- * @throws InvalidStateException
- * @throws ProvisionedThroughputException
- * @throws KinesisClientLibIOException
- */
- private synchronized void cleanupLeasesOfFinishedShards(Collection currentLeases,
- Map shardIdToShardMap, Map> shardIdToChildShardIdsMap,
- List trackedLeases, ILeaseManager leaseManager)
- throws DependencyException, InvalidStateException, ProvisionedThroughputException,
- KinesisClientLibIOException {
- Set shardIdsOfClosedShards = new HashSet<>();
- List leasesOfClosedShards = new ArrayList<>();
- for (KinesisClientLease lease : currentLeases) {
- if (lease.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END)) {
- shardIdsOfClosedShards.add(lease.getLeaseKey());
- leasesOfClosedShards.add(lease);
- }
- }
-
- if (!leasesOfClosedShards.isEmpty()) {
- assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, shardIdsOfClosedShards);
- Comparator super KinesisClientLease> startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator(
- shardIdToShardMap);
- Collections.sort(leasesOfClosedShards, startingSequenceNumberComparator);
- Map trackedLeaseMap = constructShardIdToKCLLeaseMap(trackedLeases);
-
- for (KinesisClientLease leaseOfClosedShard : leasesOfClosedShards) {
- String closedShardId = leaseOfClosedShard.getLeaseKey();
- Set childShardIds = shardIdToChildShardIdsMap.get(closedShardId);
- if ((closedShardId != null) && (childShardIds != null) && (!childShardIds.isEmpty())) {
- cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
- }
- }
- }
- }
-
- /**
- * 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->KinesisClientLease map with all leases we are tracking (should not be null)
- * @param leaseManager
- * @throws ProvisionedThroughputException
- * @throws InvalidStateException
- * @throws DependencyException
- */
- synchronized void cleanupLeaseForClosedShard(String closedShardId, Set childShardIds,
- Map trackedLeases, ILeaseManager leaseManager)
- throws DependencyException, InvalidStateException, ProvisionedThroughputException {
- KinesisClientLease leaseForClosedShard = trackedLeases.get(closedShardId);
- List childShardLeases = new ArrayList<>();
-
- for (String childShardId : childShardIds) {
- KinesisClientLease childLease = trackedLeases.get(childShardId);
- if (childLease != null) {
- childShardLeases.add(childLease);
- }
- }
-
- if ((leaseForClosedShard != null) && (leaseForClosedShard.getCheckpoint()
- .equals(ExtendedSequenceNumber.SHARD_END)) && (childShardLeases.size() == childShardIds.size())) {
- boolean okayToDelete = true;
- for (KinesisClientLease lease : childShardLeases) {
- if (lease.getCheckpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON)) {
- okayToDelete = false;
- break;
- }
- }
-
- if (okayToDelete) {
- LOG.info("Deleting lease for shard " + leaseForClosedShard.getLeaseKey()
- + " as it has been completely processed and processing of child shards has begun.");
- leaseManager.deleteLease(leaseForClosedShard);
- }
- }
- }
-
/**
* Helper method to create a new KinesisClientLease POJO for a shard.
* Note: Package level access only for testing purposes
@@ -747,7 +605,7 @@ class KinesisShardSyncer implements ShardSyncer {
* @param shard
* @return
*/
- KinesisClientLease newKCLLease(Shard shard) {
+ static KinesisClientLease newKCLLease(Shard shard) {
KinesisClientLease newLease = new KinesisClientLease();
newLease.setLeaseKey(shard.getShardId());
List parentShardIds = new ArrayList(2);
@@ -763,13 +621,36 @@ class KinesisShardSyncer implements ShardSyncer {
return newLease;
}
+ /**
+ * Helper method to create a new KinesisClientLease POJO for a ChildShard.
+ * Note: Package level access only for testing purposes
+ *
+ * @param childShard
+ * @return
+ */
+ static KinesisClientLease newKCLLeaseForChildShard(ChildShard childShard) throws InvalidStateException {
+ final KinesisClientLease newLease = new KinesisClientLease();
+ newLease.setLeaseKey(childShard.getShardId());
+ final List parentShardIds = new ArrayList<>();
+ if (!CollectionUtils.isNullOrEmpty(childShard.getParentShards())) {
+ parentShardIds.addAll(childShard.getParentShards());
+ } else {
+ throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.getShardId()
+ + " because parent shards cannot be found.");
+ }
+ newLease.setParentShardIds(parentShardIds);
+ newLease.setOwnerSwitchesSinceCheckpoint(0L);
+ newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
+ return newLease;
+ }
+
/**
* Helper method to construct a shardId->Shard map for the specified list of shards.
*
* @param shards List of shards
* @return ShardId->Shard map
*/
- Map constructShardIdToShardMap(List shards) {
+ static Map constructShardIdToShardMap(List shards) {
Map shardIdToShardMap = new HashMap();
for (Shard shard : shards) {
shardIdToShardMap.put(shard.getShardId(), shard);
@@ -784,7 +665,7 @@ class KinesisShardSyncer implements ShardSyncer {
* @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.
*/
- List getOpenShards(List allShards) {
+ static List getOpenShards(List allShards) {
List openShards = new ArrayList();
for (Shard shard : allShards) {
String endingSequenceNumber = shard.getSequenceNumberRange().getEndingSequenceNumber();
@@ -796,7 +677,7 @@ class KinesisShardSyncer implements ShardSyncer {
return openShards;
}
- private ExtendedSequenceNumber convertToCheckpoint(InitialPositionInStreamExtended position) {
+ static ExtendedSequenceNumber convertToCheckpoint(InitialPositionInStreamExtended position) {
ExtendedSequenceNumber checkpoint = null;
if (position.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)) {
@@ -813,7 +694,7 @@ class KinesisShardSyncer implements ShardSyncer {
/** Helper class to compare leases based on starting sequence number of the corresponding shards.
*
*/
- private static class StartingSequenceNumberAndShardIdBasedComparator implements Comparator,
+ static class StartingSequenceNumberAndShardIdBasedComparator implements Comparator,
Serializable {
private static final long serialVersionUID = 1L;
@@ -862,4 +743,28 @@ class KinesisShardSyncer implements ShardSyncer {
}
+ /**
+ * Helper class to pass around state between recursive traversals of shard hierarchy.
+ */
+ @NoArgsConstructor
+ static class MemoizationContext {
+ private Map isDescendantMap = new HashMap<>();
+ private Map shouldCreateLeaseMap = new HashMap<>();
+
+ Boolean isDescendant(String shardId) {
+ return isDescendantMap.get(shardId);
+ }
+
+ void setIsDescendant(String shardId, Boolean isDescendant) {
+ isDescendantMap.put(shardId, isDescendant);
+ }
+
+ Boolean shouldCreateLease(String shardId) {
+ return shouldCreateLeaseMap.computeIfAbsent(shardId, x -> Boolean.FALSE);
+ }
+
+ void setShouldCreateLease(String shardId, Boolean shouldCreateLease) {
+ shouldCreateLeaseMap.put(shardId, shouldCreateLease);
+ }
+ }
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java
index afb37112..7e07587a 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java
@@ -8,6 +8,7 @@ import java.util.Set;
/**
* Represents the class that decides if a lease is eligible for cleanup.
*/
+@Deprecated
public interface LeaseCleanupValidator {
/**
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java
new file mode 100644
index 00000000..441249d1
--- /dev/null
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java
@@ -0,0 +1,41 @@
+/*
+ * 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 com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+
+import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
+import com.amazonaws.services.kinesis.model.Shard;
+
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Interface used by {@link KinesisShardSyncer} to determine how to create new leases based on the current state
+ * of the lease table (i.e. whether the lease table is empty or non-empty).
+ */
+interface LeaseSynchronizer {
+
+ /**
+ * Determines how to create leases.
+ * @param shards
+ * @param currentLeases
+ * @param initialPosition
+ * @param inconsistentShardIds
+ * @return
+ */
+ List determineNewLeasesToCreate(List shards,
+ List currentLeases,
+ InitialPositionInStreamExtended initialPosition,
+ Set inconsistentShardIds);
+}
\ No newline at end of file
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java
new file mode 100644
index 00000000..2a868d30
--- /dev/null
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java
@@ -0,0 +1,162 @@
+/*
+ * 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 com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
+import com.amazonaws.services.kinesis.leases.impl.Lease;
+import com.amazonaws.services.kinesis.model.Shard;
+import lombok.AllArgsConstructor;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * TODO - non-empty lease table sync story
+ */
+@AllArgsConstructor
+class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer {
+
+ private static final Log LOG = LogFactory.getLog(NonEmptyLeaseTableSynchronizer.class);
+
+ 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.
+ *
+ * @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.
+ * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard
+ */
+ @Override
+ public List determineNewLeasesToCreate(List shards,
+ List currentLeases,
+ InitialPositionInStreamExtended initialPosition,
+ Set inconsistentShardIds) {
+
+ Map shardIdToNewLeaseMap = new HashMap<>();
+ Map shardIdToShardMapOfAllKinesisShards = KinesisShardSyncer.constructShardIdToShardMap(shards);
+
+ Set shardIdsOfCurrentLeases = new HashSet();
+ for (Lease lease : currentLeases) {
+ shardIdsOfCurrentLeases.add(lease.getLeaseKey());
+ LOG.debug("Existing lease: " + lease);
+ }
+
+ List openShards = KinesisShardSyncer.getOpenShards(shards);
+ final KinesisShardSyncer.MemoizationContext memoizationContext = new KinesisShardSyncer.MemoizationContext();
+
+
+ // Iterate over the open shards and find those that don't have any lease entries.
+ for (Shard shard : openShards) {
+ String shardId = shard.getShardId();
+ LOG.debug("Evaluating leases for open shard " + shardId + " and its ancestors.");
+ if (shardIdsOfCurrentLeases.contains(shardId)) {
+ LOG.debug("Lease for shardId " + shardId + " already exists. Not creating a lease");
+ } else if (inconsistentShardIds.contains(shardId)) {
+ LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease");
+ } else {
+ LOG.debug("Beginning traversal of ancestry tree for shardId " + shardId);
+
+ // A shard is a descendant if at least one if its ancestors exists in the lease table.
+ // We will create leases for only one level in the ancestry tree. Once we find the first ancestor
+ // that needs to be processed in order to complete the hash range, we will not create leases for
+ // further descendants of that ancestor.
+ boolean isDescendant = KinesisShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId,
+ initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards,
+ shardIdToNewLeaseMap, memoizationContext);
+
+ // If shard is a descendant, the leases for its ancestors were already created above. Open shards
+ // that are NOT descendants will not have leases yet, so we create them here. We will not create
+ // leases for open shards that ARE descendants yet - leases for these shards will be created upon
+ // SHARD_END of their parents.
+ if (!isDescendant) {
+ LOG.debug("ShardId " + shardId + " has no ancestors. Creating a lease.");
+ final KinesisClientLease newLease = KinesisShardSyncer.newKCLLease(shard);
+ newLease.setCheckpoint(KinesisShardSyncer.convertToCheckpoint(initialPosition));
+ LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint());
+ shardIdToNewLeaseMap.put(shardId, newLease);
+ } else {
+ LOG.debug("ShardId " + shardId + " is a descendant whose ancestors should already have leases. " +
+ "Not creating a lease.");
+ }
+ }
+ }
+
+ List newLeasesToCreate = new ArrayList<>();
+ newLeasesToCreate.addAll(shardIdToNewLeaseMap.values());
+ Comparator super KinesisClientLease> startingSequenceNumberComparator = new KinesisShardSyncer.StartingSequenceNumberAndShardIdBasedComparator(
+ shardIdToShardMapOfAllKinesisShards);
+ Collections.sort(newLeasesToCreate, startingSequenceNumberComparator);
+ return newLeasesToCreate;
+ }
+}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java
index 6359b3b1..126eb6ea 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java
@@ -25,6 +25,7 @@ import java.util.Map;
* It also limits number of shards that will be available for initialization based on their depth.
* It doesn't make a lot of sense to work on a shard that has too many unfinished parents.
*/
+@Deprecated
public class ParentsFirstShardPrioritization implements
ShardPrioritization {
private static final SortingNode PROCESSING_NODE = new SortingNode(null, Integer.MIN_VALUE);
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java
index d129944f..cdf73e82 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java
@@ -14,39 +14,105 @@
*/
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import com.amazonaws.services.cloudwatch.model.StandardUnit;
+import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy;
+import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
+import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
+import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease;
+import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
+import com.amazonaws.services.kinesis.leases.impl.UpdateField;
+import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
+import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper;
import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
+import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.util.CollectionUtils;
+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 org.apache.commons.lang3.Validate;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.fromHashKeyRange;
+
/**
- * The top level orchestrator for coordinating the periodic shard sync related
- * activities.
+ * The top level orchestrator for coordinating the periodic shard sync related activities. If the configured
+ * {@link ShardSyncStrategyType} is PERIODIC, this class will be the main shard sync orchestrator. For non-PERIODIC
+ * strategies, this class will serve as an internal auditor that periodically checks if the full hash range is covered
+ * by currently held leases, and initiates a recovery shard sync if not.
*/
@Getter
@EqualsAndHashCode
class PeriodicShardSyncManager {
private static final Log LOG = LogFactory.getLog(PeriodicShardSyncManager.class);
private static final long INITIAL_DELAY = 0;
- private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000;
+
+ /** DEFAULT interval is used for PERIODIC {@link ShardSyncStrategyType}. */
+ private static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000L;
+
+ /** Parameters for validating hash range completeness when running in auditor mode. */
+ @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 final HashRangeHoleTracker hashRangeHoleTracker = new HashRangeHoleTracker();
private final String workerId;
private final LeaderDecider leaderDecider;
private final ITask metricsEmittingShardSyncTask;
private final ScheduledExecutorService shardSyncThreadPool;
+ private final ILeaseManager leaseManager;
+ private final IKinesisProxy kinesisProxy;
+ private final boolean isAuditorMode;
+ private final long periodicShardSyncIntervalMillis;
private boolean isRunning;
+ private final IMetricsFactory metricsFactory;
+ private final int leasesRecoveryAuditorInconsistencyConfidenceThreshold;
- PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, IMetricsFactory metricsFactory) {
- this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory);
+
+ PeriodicShardSyncManager(String workerId,
+ LeaderDecider leaderDecider,
+ ShardSyncTask shardSyncTask,
+ IMetricsFactory metricsFactory,
+ ILeaseManager leaseManager,
+ IKinesisProxy kinesisProxy,
+ boolean isAuditorMode,
+ long leasesRecoveryAuditorExecutionFrequencyMillis,
+ int leasesRecoveryAuditorInconsistencyConfidenceThreshold) {
+ this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory,
+ leaseManager, kinesisProxy, isAuditorMode, leasesRecoveryAuditorExecutionFrequencyMillis,
+ leasesRecoveryAuditorInconsistencyConfidenceThreshold);
}
- PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, ScheduledExecutorService shardSyncThreadPool, IMetricsFactory metricsFactory) {
+ PeriodicShardSyncManager(String workerId,
+ LeaderDecider leaderDecider,
+ ShardSyncTask shardSyncTask,
+ ScheduledExecutorService shardSyncThreadPool,
+ IMetricsFactory metricsFactory,
+ ILeaseManager leaseManager,
+ IKinesisProxy kinesisProxy,
+ boolean isAuditorMode,
+ long leasesRecoveryAuditorExecutionFrequencyMillis,
+ int leasesRecoveryAuditorInconsistencyConfidenceThreshold) {
Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager.");
Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager.");
Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager.");
@@ -54,18 +120,47 @@ class PeriodicShardSyncManager {
this.leaderDecider = leaderDecider;
this.metricsEmittingShardSyncTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory);
this.shardSyncThreadPool = shardSyncThreadPool;
+ this.leaseManager = leaseManager;
+ this.kinesisProxy = kinesisProxy;
+ this.metricsFactory = metricsFactory;
+ this.isAuditorMode = isAuditorMode;
+ this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold;
+ if (isAuditorMode) {
+ Validate.notNull(this.leaseManager, "LeaseManager is required for non-PERIODIC shard sync strategies.");
+ Validate.notNull(this.kinesisProxy, "KinesisProxy is required for non-PERIODIC shard sync strategies.");
+ this.periodicShardSyncIntervalMillis = leasesRecoveryAuditorExecutionFrequencyMillis;
+ } else {
+ this.periodicShardSyncIntervalMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS;
+ }
}
public synchronized TaskResult start() {
if (!isRunning) {
+ final Runnable periodicShardSyncer = () -> {
+ try {
+ runShardSync();
+ } catch (Throwable t) {
+ LOG.error("Error running shard sync.", t);
+ }
+ };
+
shardSyncThreadPool
- .scheduleWithFixedDelay(this::runShardSync, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS,
+ .scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, periodicShardSyncIntervalMillis,
TimeUnit.MILLISECONDS);
isRunning = true;
}
return new TaskResult(null);
}
+ /**
+ * Runs ShardSync once, without scheduling further periodic ShardSyncs.
+ * @return TaskResult from shard sync
+ */
+ public synchronized TaskResult syncShardsOnce() {
+ LOG.info("Syncing shards once from worker " + workerId);
+ return metricsEmittingShardSyncTask.call();
+ }
+
public void stop() {
if (isRunning) {
LOG.info(String.format("Shutting down leader decider on worker %s", workerId));
@@ -77,15 +172,239 @@ class PeriodicShardSyncManager {
}
private void runShardSync() {
- try {
- if (leaderDecider.isLeader(workerId)) {
- LOG.debug(String.format("WorkerId %s is a leader, running the shard sync task", workerId));
- metricsEmittingShardSyncTask.call();
- } else {
- LOG.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId));
+ if (leaderDecider.isLeader(workerId)) {
+ LOG.debug("WorkerId " + workerId + " is a leader, running the shard sync task");
+
+ MetricsHelper.startScope(metricsFactory, PERIODIC_SHARD_SYNC_MANAGER);
+ boolean isRunSuccess = false;
+ final long runStartMillis = System.currentTimeMillis();
+
+ try {
+ final ShardSyncResponse shardSyncResponse = checkForShardSync();
+ MetricsHelper.getMetricsScope().addData("NumStreamsToSync", shardSyncResponse.shouldDoShardSync() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY);
+ MetricsHelper.getMetricsScope().addData("NumStreamsWithPartialLeases", shardSyncResponse.isHoleDetected() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY);
+ if (shardSyncResponse.shouldDoShardSync()) {
+ LOG.info("Periodic shard syncer initiating shard sync due to the reason - " +
+ shardSyncResponse.reasonForDecision());
+ metricsEmittingShardSyncTask.call();
+ } else {
+ LOG.info("Skipping shard sync due to the reason - " + shardSyncResponse.reasonForDecision());
+ }
+ isRunSuccess = true;
+ } catch (Exception e) {
+ LOG.error("Caught exception while running periodic shard syncer.", e);
+ } finally {
+ MetricsHelper.addSuccessAndLatency(runStartMillis, isRunSuccess, MetricsLevel.SUMMARY);
+ MetricsHelper.endScope();
}
- } catch (Throwable t) {
- LOG.error("Error during runShardSync.", t);
+ } else {
+ LOG.debug("WorkerId " + workerId + " is not a leader, not running the shard sync task");
+ }
+ }
+
+ @VisibleForTesting
+ ShardSyncResponse checkForShardSync() throws DependencyException, InvalidStateException,
+ ProvisionedThroughputException {
+
+ if (!isAuditorMode) {
+ // If we are running with PERIODIC shard sync strategy, we should sync every time.
+ return new ShardSyncResponse(true, false, "Syncing every time with PERIODIC shard sync strategy.");
+ }
+
+ // Get current leases from DynamoDB.
+ final List currentLeases = leaseManager.listLeases();
+
+ if (CollectionUtils.isNullOrEmpty(currentLeases)) {
+ // If the current leases are null or empty, then we need to initiate a shard sync.
+ LOG.info("No leases found. Will trigger a shard sync.");
+ return new ShardSyncResponse(true, false, "No leases found.");
+ }
+
+ // Check if there are any holes in the hash range covered by current leases. Return the first hole if present.
+ Optional hashRangeHoleOpt = hasHoleInLeases(currentLeases);
+ 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 use the high confidence factor
+ // to avoid shard sync on any holes during resharding and lease cleanups, or other intermittent issues.
+ final boolean hasHoleWithHighConfidence =
+ hashRangeHoleTracker.hashHighConfidenceOfHoleWith(hashRangeHoleOpt.get());
+
+ return new ShardSyncResponse(hasHoleWithHighConfidence, true,
+ "Detected the same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + " times. " +
+ "Will initiate shard sync after reaching threshold: " + leasesRecoveryAuditorInconsistencyConfidenceThreshold);
+ } else {
+ // If hole is not present, clear any previous hole tracking and return false.
+ hashRangeHoleTracker.reset();
+ return new ShardSyncResponse(false, false, "Hash range is complete.");
+ }
+ }
+
+ @VisibleForTesting
+ Optional hasHoleInLeases(List leases) {
+ // Filter out any leases with checkpoints other than SHARD_END
+ final List activeLeases = leases.stream()
+ .filter(lease -> lease.getCheckpoint() != null && !lease.getCheckpoint().isShardEnd())
+ .collect(Collectors.toList());
+
+ final List activeLeasesWithHashRanges = fillWithHashRangesIfRequired(activeLeases);
+ return checkForHoleInHashKeyRanges(activeLeasesWithHashRanges);
+ }
+
+ private List fillWithHashRangesIfRequired(List activeLeases) {
+ final List activeLeasesWithNoHashRanges = activeLeases.stream()
+ .filter(lease -> lease.getHashKeyRange() == null).collect(Collectors.toList());
+
+ if (activeLeasesWithNoHashRanges.isEmpty()) {
+ return activeLeases;
+ }
+
+ // Fetch shards from Kinesis to fill in the in-memory hash ranges
+ final Map kinesisShards = kinesisProxy.getShardList().stream()
+ .collect(Collectors.toMap(Shard::getShardId, shard -> shard));
+
+ return activeLeases.stream().map(lease -> {
+ if (lease.getHashKeyRange() == null) {
+ final String shardId = lease.getLeaseKey();
+ final Shard shard = kinesisShards.get(shardId);
+ if (shard == null) {
+ return lease;
+ }
+ lease.setHashKeyRange(fromHashKeyRange(shard.getHashKeyRange()));
+
+ try {
+ leaseManager.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE);
+ } catch (Exception e) {
+ LOG.warn("Unable to update hash range information for lease " + lease.getLeaseKey() +
+ ". This may result in explicit lease sync.");
+ }
+ }
+ return lease;
+ }).filter(lease -> lease.getHashKeyRange() != null).collect(Collectors.toList());
+ }
+
+ @VisibleForTesting
+ static Optional checkForHoleInHashKeyRanges(List leasesWithHashKeyRanges) {
+ // Sort the hash ranges by starting hash key
+ final List sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges);
+ if (sortedLeasesWithHashKeyRanges.isEmpty()) {
+ LOG.error("No leases with valid hash ranges found.");
+ return Optional.of(new HashRangeHole());
+ }
+
+ // Validate the hash range bounds
+ final KinesisClientLease minHashKeyLease = sortedLeasesWithHashKeyRanges.get(0);
+ final KinesisClientLease maxHashKeyLease =
+ sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1);
+ if (!minHashKeyLease.getHashKeyRange().startingHashKey().equals(MIN_HASH_KEY) ||
+ !maxHashKeyLease.getHashKeyRange().endingHashKey().equals(MAX_HASH_KEY)) {
+ LOG.error("Incomplete hash range found between " + minHashKeyLease + " and " + maxHashKeyLease);
+ return Optional.of(new HashRangeHole(minHashKeyLease.getHashKeyRange(), maxHashKeyLease.getHashKeyRange()));
+ }
+
+ // Check for any holes in the sorted hash range intervals
+ if (sortedLeasesWithHashKeyRanges.size() > 1) {
+ KinesisClientLease leftmostLeaseToReportInCaseOfHole = minHashKeyLease;
+ HashKeyRangeForLease leftLeaseHashRange = leftmostLeaseToReportInCaseOfHole.getHashKeyRange();
+
+ for (int i = 1; i < sortedLeasesWithHashKeyRanges.size(); i++) {
+ final KinesisClientLease rightLease = sortedLeasesWithHashKeyRanges.get(i);
+ final HashKeyRangeForLease rightLeaseHashRange = rightLease.getHashKeyRange();
+ final BigInteger rangeDiff =
+ rightLeaseHashRange.startingHashKey().subtract(leftLeaseHashRange.endingHashKey());
+ // We have overlapping leases when rangeDiff is 0 or negative.
+ // signum() will be -1 for negative and 0 if value is 0.
+ // Merge the ranges for further tracking.
+ if (rangeDiff.signum() <= 0) {
+ leftLeaseHashRange = new HashKeyRangeForLease(leftLeaseHashRange.startingHashKey(),
+ leftLeaseHashRange.endingHashKey().max(rightLeaseHashRange.endingHashKey()));
+ } else {
+ // We have non-overlapping leases when rangeDiff is positive. signum() will be 1 in this case.
+ // If rangeDiff is 1, then it is a continuous hash range. If not, there is a hole.
+ if (!rangeDiff.equals(BigInteger.ONE)) {
+ LOG.error("Incomplete hash range found between " + leftmostLeaseToReportInCaseOfHole +
+ " and " + rightLease);
+ return Optional.of(new HashRangeHole(leftmostLeaseToReportInCaseOfHole.getHashKeyRange(),
+ rightLease.getHashKeyRange()));
+ }
+
+ leftmostLeaseToReportInCaseOfHole = rightLease;
+ 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
+ @Accessors(fluent = true)
+ @VisibleForTesting
+ static class ShardSyncResponse {
+ private final boolean shouldDoShardSync;
+ private final boolean isHoleDetected;
+ private final String reasonForDecision;
+ }
+
+ @Value
+ private static class HashRangeHole {
+ private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole;
+ private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole;
+
+ HashRangeHole() {
+ hashRangeAtStartOfPossibleHole = hashRangeAtEndOfPossibleHole = null;
+ }
+
+ HashRangeHole(HashKeyRangeForLease hashRangeAtStartOfPossibleHole,
+ HashKeyRangeForLease hashRangeAtEndOfPossibleHole) {
+ this.hashRangeAtStartOfPossibleHole = hashRangeAtStartOfPossibleHole;
+ this.hashRangeAtEndOfPossibleHole = hashRangeAtEndOfPossibleHole;
+ }
+ }
+
+ private class HashRangeHoleTracker {
+ private HashRangeHole hashRangeHole;
+ @Getter
+ private Integer numConsecutiveHoles;
+
+ public boolean hashHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) {
+ if (hashRangeHole.equals(this.hashRangeHole)) {
+ ++this.numConsecutiveHoles;
+ } else {
+ this.hashRangeHole = hashRangeHole;
+ this.numConsecutiveHoles = 1;
+ }
+
+ return numConsecutiveHoles >= leasesRecoveryAuditorInconsistencyConfidenceThreshold;
+ }
+
+ public void reset() {
+ this.hashRangeHole = null;
+ this.numConsecutiveHoles = 0;
+ }
+ }
+
+ private static class HashKeyRangeComparator implements Comparator, Serializable {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public int compare(KinesisClientLease lease, KinesisClientLease otherLease) {
+ Validate.notNull(lease);
+ Validate.notNull(otherLease);
+ Validate.notNull(lease.getHashKeyRange());
+ Validate.notNull(otherLease.getHashKeyRange());
+ return ComparisonChain.start()
+ .compare(lease.getHashKeyRange().startingHashKey(), otherLease.getHashKeyRange().startingHashKey())
+ .compare(lease.getHashKeyRange().endingHashKey(), otherLease.getHashKeyRange().endingHashKey())
+ .result();
}
}
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java
index 8173a479..a4cf74d8 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java
@@ -129,6 +129,7 @@ public class PrefetchGetRecordsCache implements GetRecordsCache {
try {
result = getRecordsResultQueue.take().withCacheExitTime(Instant.now());
prefetchCounters.removed(result);
+ log.info("Shard " + shardId + ": Number of records remaining in queue is " + getRecordsResultQueue.size());
} catch (InterruptedException e) {
log.error("Interrupted while getting records from the cache", e);
}
@@ -177,7 +178,6 @@ public class PrefetchGetRecordsCache implements GetRecordsCache {
MetricsHelper.getMetricsScope().addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.Count,
MetricsLevel.SUMMARY);
-
dataFetcher.restartIterator();
} catch (SdkClientException e) {
log.error("Exception thrown while fetching records from Kinesis", e);
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java
index b578fbb0..cd543e23 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java
@@ -152,8 +152,8 @@ class ProcessTask implements ITask {
try {
if (dataFetcher.isShardEndReached()) {
- LOG.info("Reached end of shard " + shardInfo.getShardId());
- return new TaskResult(null, true);
+ LOG.info("Reached end of shard " + shardInfo.getShardId() + ". Found childShards: " + dataFetcher.getChildShards());
+ return new TaskResult(null, true, dataFetcher.getChildShards());
}
final ProcessRecordsInput processRecordsInput = getRecordsResult();
@@ -353,7 +353,7 @@ class ProcessTask implements ITask {
* recordProcessorCheckpointer).
*/
dataFetcher.advanceIteratorTo(recordProcessorCheckpointer.getLargestPermittedCheckpointValue()
- .getSequenceNumber(), streamConfig.getInitialPositionInStream());
+ .getSequenceNumber(), streamConfig.getInitialPositionInStream());
// Try a second time - if we fail this time, expose the failure.
try {
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java
index a30412ce..29a95ac4 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java
@@ -15,11 +15,16 @@
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+import java.util.List;
import java.util.Optional;
import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.RejectedExecutionException;
+import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager;
+import com.amazonaws.services.kinesis.model.ChildShard;
+import com.amazonaws.util.CollectionUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -52,6 +57,7 @@ class ShardConsumer {
private final IMetricsFactory metricsFactory;
private final KinesisClientLibLeaseCoordinator leaseCoordinator;
private ICheckpoint checkpoint;
+ private LeaseCleanupManager leaseCleanupManager;
// Backoff time when polling to check if application has finished processing parent shards
private final long parentShardPollIntervalMillis;
private final boolean cleanupLeasesOfCompletedShards;
@@ -66,6 +72,9 @@ class ShardConsumer {
private Future future;
private ShardSyncStrategy shardSyncStrategy;
+ @Getter
+ private List childShards;
+
@Getter
private final GetRecordsCache getRecordsCache;
@@ -106,6 +115,7 @@ class ShardConsumer {
* @param shardSyncer shardSyncer instance used to check and create new leases
*/
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
+ @Deprecated
ShardConsumer(ShardInfo shardInfo,
StreamConfig streamConfig,
ICheckpoint checkpoint,
@@ -118,6 +128,7 @@ class ShardConsumer {
long backoffTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
KinesisClientLibConfiguration config, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) {
+
this(shardInfo,
streamConfig,
checkpoint,
@@ -150,6 +161,7 @@ class ShardConsumer {
* @param shardSyncer shardSyncer instance used to check and create new leases
*/
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
+ @Deprecated
ShardConsumer(ShardInfo shardInfo,
StreamConfig streamConfig,
ICheckpoint checkpoint,
@@ -210,6 +222,7 @@ class ShardConsumer {
* @param config Kinesis library configuration
* @param shardSyncer shardSyncer instance used to check and create new leases
*/
+ @Deprecated
ShardConsumer(ShardInfo shardInfo,
StreamConfig streamConfig,
ICheckpoint checkpoint,
@@ -226,6 +239,53 @@ class ShardConsumer {
Optional retryGetRecordsInSeconds,
Optional maxGetRecordsThreadPool,
KinesisClientLibConfiguration config, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) {
+
+ this(shardInfo, streamConfig, checkpoint, recordProcessor, recordProcessorCheckpointer, leaseCoordinator,
+ parentShardPollIntervalMillis, cleanupLeasesOfCompletedShards, executorService, metricsFactory,
+ backoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, kinesisDataFetcher, retryGetRecordsInSeconds,
+ maxGetRecordsThreadPool, config, shardSyncer, shardSyncStrategy, LeaseCleanupManager.createOrGetInstance(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(),
+ Executors.newSingleThreadScheduledExecutor(), metricsFactory, config.shouldCleanupLeasesUponShardCompletion(),
+ config.leaseCleanupIntervalMillis(), config.completedLeaseCleanupThresholdMillis(),
+ config.garbageLeaseCleanupThresholdMillis(), config.getMaxRecords()));
+ }
+
+ /**
+ * @param shardInfo Shard information
+ * @param streamConfig Stream Config to use
+ * @param checkpoint Checkpoint tracker
+ * @param recordProcessor Record processor used to process the data records for the shard
+ * @param recordProcessorCheckpointer RecordProcessorCheckpointer to use to checkpoint progress
+ * @param leaseCoordinator Used to manage leases for current worker
+ * @param parentShardPollIntervalMillis Wait for this long if parent shards are not done (or we get an exception)
+ * @param cleanupLeasesOfCompletedShards clean up the leases of completed shards
+ * @param executorService ExecutorService used to execute process tasks for this shard
+ * @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard
+ * @param backoffTimeMillis backoff interval when we encounter exceptions
+ * @param skipShardSyncAtWorkerInitializationIfLeasesExist Skip sync at init if lease exists
+ * @param kinesisDataFetcher KinesisDataFetcher to fetch data from Kinesis streams.
+ * @param retryGetRecordsInSeconds time in seconds to wait before the worker retries to get a record
+ * @param maxGetRecordsThreadPool max number of threads in the getRecords thread pool
+ * @param config Kinesis library configuration
+ * @param shardSyncer shardSyncer instance used to check and create new leases
+ * @param leaseCleanupManager used to clean up leases in lease table.
+ */
+ ShardConsumer(ShardInfo shardInfo,
+ StreamConfig streamConfig,
+ ICheckpoint checkpoint,
+ IRecordProcessor recordProcessor,
+ RecordProcessorCheckpointer recordProcessorCheckpointer,
+ KinesisClientLibLeaseCoordinator leaseCoordinator,
+ long parentShardPollIntervalMillis,
+ boolean cleanupLeasesOfCompletedShards,
+ ExecutorService executorService,
+ IMetricsFactory metricsFactory,
+ long backoffTimeMillis,
+ boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
+ KinesisDataFetcher kinesisDataFetcher,
+ Optional retryGetRecordsInSeconds,
+ Optional maxGetRecordsThreadPool,
+ KinesisClientLibConfiguration config, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy,
+ LeaseCleanupManager leaseCleanupManager) {
this.shardInfo = shardInfo;
this.streamConfig = streamConfig;
this.checkpoint = checkpoint;
@@ -245,6 +305,7 @@ class ShardConsumer {
this.getShardInfo().getShardId(), this.metricsFactory, this.config.getMaxRecords());
this.shardSyncer = shardSyncer;
this.shardSyncStrategy = shardSyncStrategy;
+ this.leaseCleanupManager = leaseCleanupManager;
}
/**
@@ -321,6 +382,10 @@ class ShardConsumer {
TaskResult result = future.get();
if (result.getException() == null) {
if (result.isShardEndReached()) {
+ if (!CollectionUtils.isNullOrEmpty(result.getChildShards())) {
+ childShards = result.getChildShards();
+ LOG.info("Shard " + shardInfo.getShardId() + ": Setting childShards in ShardConsumer: " + childShards);
+ }
return TaskOutcome.END_OF_SHARD;
}
return TaskOutcome.SUCCESSFUL;
@@ -420,6 +485,7 @@ class ShardConsumer {
void updateState(TaskOutcome taskOutcome) {
if (taskOutcome == TaskOutcome.END_OF_SHARD) {
markForShutdown(ShutdownReason.TERMINATE);
+ LOG.info("Shard " + shardInfo.getShardId() + ": Mark for shutdown with reason TERMINATE");
}
if (isShutdownRequested() && taskOutcome != TaskOutcome.FAILURE) {
currentState = currentState.shutdownTransition(shutdownReason);
@@ -518,4 +584,8 @@ class ShardConsumer {
ShardSyncStrategy getShardSyncStrategy() {
return shardSyncStrategy;
}
+
+ LeaseCleanupManager getLeaseCleanupManager() {
+ return leaseCleanupManager;
+ }
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java
index 8077efcc..9efe2f51 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java
@@ -16,8 +16,13 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy {
private static final Log LOG = LogFactory.getLog(Worker.class);
private ShardSyncTaskManager shardSyncTaskManager;
- ShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager) {
+ /** Runs periodic shard sync jobs in the background as an auditor process for shard-end syncs. */
+ private PeriodicShardSyncManager periodicShardSyncManager;
+
+ ShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager,
+ PeriodicShardSyncManager periodicShardSyncManager) {
this.shardSyncTaskManager = shardSyncTaskManager;
+ this.periodicShardSyncManager = periodicShardSyncManager;
}
@Override
@@ -42,8 +47,8 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy {
@Override
public TaskResult onWorkerInitialization() {
- LOG.debug(String.format("onWorkerInitialization is NoOp for ShardSyncStrategyType %s", getStrategyType().toString()));
- return new TaskResult(null);
+ LOG.info("Starting periodic shard sync background process for SHARD_END shard sync strategy.");
+ return periodicShardSyncManager.start();
}
@Override
@@ -65,6 +70,7 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy {
@Override
public void onWorkerShutDown() {
- LOG.debug(String.format("Stop is NoOp for ShardSyncStrategyType %s", getStrategyType().toString()));
+ LOG.info("Stopping periodic shard sync background process for SHARD_END shard sync strategy.");
+ periodicShardSyncManager.stop();
}
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java
index 24b30e69..3b04b791 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java
@@ -86,7 +86,7 @@ public class ShardInfo {
*
* @return a list of shardId's that are parents of this shard, or empty if the shard has no parents.
*/
- protected List getParentShardIds() {
+ public List getParentShardIds() {
return new LinkedList(parentShardIds);
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java
index a9ff5080..274aaaa1 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java
@@ -14,9 +14,16 @@
*/
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
-import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardClosureVerificationResponse;
-import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse;
-import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.leases.LeasePendingDeletion;
+import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.BlockedOnParentShardException;
+import com.amazonaws.services.kinesis.leases.exceptions.CustomerApplicationException;
+import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
+import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
+import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager;
+import com.amazonaws.services.kinesis.leases.impl.UpdateField;
+import com.amazonaws.services.kinesis.model.ChildShard;
+import com.amazonaws.util.CollectionUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -25,11 +32,14 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy;
import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
-import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper;
-import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel;
import com.google.common.annotations.VisibleForTesting;
import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
/**
* Task for invoking the RecordProcessor shutdown() callback.
@@ -38,7 +48,8 @@ class ShutdownTask implements ITask {
private static final Log LOG = LogFactory.getLog(ShutdownTask.class);
- private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown";
+ @VisibleForTesting
+ static final int RETRY_RANDOM_MAX_RANGE = 50;
private final ShardInfo shardInfo;
private final IRecordProcessor recordProcessor;
@@ -54,6 +65,8 @@ class ShutdownTask implements ITask {
private final GetRecordsCache getRecordsCache;
private final ShardSyncer shardSyncer;
private final ShardSyncStrategy shardSyncStrategy;
+ private final List childShards;
+ private final LeaseCleanupManager leaseCleanupManager;
/**
* Constructor.
@@ -69,7 +82,9 @@ class ShutdownTask implements ITask {
boolean ignoreUnexpectedChildShards,
KinesisClientLibLeaseCoordinator leaseCoordinator,
long backoffTimeMillis,
- GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) {
+ GetRecordsCache getRecordsCache, ShardSyncer shardSyncer,
+ ShardSyncStrategy shardSyncStrategy, List childShards,
+ LeaseCleanupManager leaseCleanupManager) {
this.shardInfo = shardInfo;
this.recordProcessor = recordProcessor;
this.recordProcessorCheckpointer = recordProcessorCheckpointer;
@@ -83,6 +98,8 @@ class ShutdownTask implements ITask {
this.getRecordsCache = getRecordsCache;
this.shardSyncer = shardSyncer;
this.shardSyncStrategy = shardSyncStrategy;
+ this.childShards = childShards;
+ this.leaseCleanupManager = leaseCleanupManager;
}
/*
@@ -94,87 +111,40 @@ class ShutdownTask implements ITask {
@Override
public TaskResult call() {
Exception exception;
- boolean applicationException = false;
+
+ LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: "
+ + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards);
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.TERMINATE) {
- ShardClosureVerificationResponse shardClosureVerificationResponse = kinesisProxy.verifyShardClosure(shardInfo.getShardId());
- if (shardClosureVerificationResponse instanceof ShardListWrappingShardClosureVerificationResponse) {
- latestShards = ((ShardListWrappingShardClosureVerificationResponse)shardClosureVerificationResponse).getLatestShards();
- }
+ final KinesisClientLease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId());
+ final Runnable leaseLostAction = () -> takeLeaseLostAction();
- // If shard in context is not closed yet we should shut down the ShardConsumer with Zombie state
- // which avoids checkpoint-ing with SHARD_END sequence number.
- if(!shardClosureVerificationResponse.isShardClosed()) {
- localReason = ShutdownReason.ZOMBIE;
- dropLease();
- LOG.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfo.getShardId());
+ if (reason == ShutdownReason.TERMINATE) {
+ try {
+ takeShardEndAction(currentShardLease);
+ } 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 ZOMBIE reason to allow other worker to take the lease and retry shutting down.
+ LOG.warn("Lease " + shardInfo.getShardId() + ": Invalid state encountered while shutting down shardConsumer with TERMINATE reason. " +
+ "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. ", e);
+ dropLease(currentShardLease);
+ throwOnApplicationException(leaseLostAction);
}
+ } else {
+ throwOnApplicationException(leaseLostAction);
}
-
- // If we reached end of the shard, set sequence number to SHARD_END.
- if (localReason == ShutdownReason.TERMINATE) {
- recordProcessorCheckpointer.setSequenceNumberAtShardEnd(
- recordProcessorCheckpointer.getLargestPermittedCheckpointValue());
- recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END);
- }
-
- LOG.debug("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken "
- + shardInfo.getConcurrencyToken() + ". Shutdown reason: " + localReason);
- final ShutdownInput shutdownInput = new ShutdownInput()
- .withShutdownReason(localReason)
- .withCheckpointer(recordProcessorCheckpointer);
- final long recordProcessorStartTimeMillis = System.currentTimeMillis();
- try {
- recordProcessor.shutdown(shutdownInput);
- ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.getLastCheckpointValue();
-
- if (localReason == ShutdownReason.TERMINATE) {
- if ((lastCheckpointValue == null)
- || (!lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END))) {
- throw new IllegalArgumentException("Application didn't checkpoint at end of shard "
- + shardInfo.getShardId() + ". Application must checkpoint upon shutdown. " +
- "See IRecordProcessor.shutdown javadocs for more information.");
- }
- }
- LOG.debug("Shutting down retrieval strategy.");
- getRecordsCache.shutdown();
- LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId());
- } catch (Exception e) {
- applicationException = true;
- throw e;
- } finally {
- MetricsHelper.addLatency(RECORD_PROCESSOR_SHUTDOWN_METRIC, recordProcessorStartTimeMillis,
- MetricsLevel.SUMMARY);
- }
-
- if (localReason == ShutdownReason.TERMINATE) {
- LOG.debug("Looking for child shards of shard " + shardInfo.getShardId());
- // create leases for the child shards
- TaskResult result = shardSyncStrategy.onShardConsumerShutDown(latestShards);
- if (result.getException() != null) {
- LOG.debug("Exception while trying to sync shards on the shutdown of shard: " + shardInfo
- .getShardId());
- throw result.getException();
- }
- LOG.debug("Finished checking for child shards of shard " + shardInfo.getShardId());
- }
-
+ LOG.debug("Shutting down retrieval strategy.");
+ getRecordsCache.shutdown();
+ LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId());
return new TaskResult(null);
} catch (Exception e) {
- if (applicationException) {
- LOG.error("Application exception. ", e);
+ if (e instanceof CustomerApplicationException) {
+ LOG.error("Shard " + shardInfo.getShardId() + ": Application exception: ", e);
} else {
- LOG.error("Caught exception: ", e);
+ LOG.error("Shard " + shardInfo.getShardId() + ": Caught exception: ", e);
}
+
exception = e;
// backoff if we encounter an exception.
try {
@@ -187,6 +157,143 @@ class ShutdownTask implements ITask {
return new TaskResult(exception);
}
+ // Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup.
+ private void takeShardEndAction(KinesisClientLease currentShardLease)
+ throws InvalidStateException, DependencyException, ProvisionedThroughputException, 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 catches ResourceNotFound exception.
+ // In this case, KinesisDataFetcher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards.
+ // This scenario could happen when customer deletes the stream while leaving the KCL application running.
+ if (currentShardLease == null) {
+ throw new InvalidStateException("Shard " + shardInfo.getShardId() + ": Lease not owned by the current worker. Leaving ShardEnd handling to new owner.");
+ }
+ if (!CollectionUtils.isNullOrEmpty(childShards)) {
+ // If childShards is not empty, create new leases for the childShards and update the current lease with the childShards lease information.
+ createLeasesForChildShardsIfNotExist();
+ updateCurrentLeaseWithChildShards(currentShardLease);
+ } else {
+ LOG.warn("Shard " + shardInfo.getShardId()
+ + ": Shutting down consumer with SHARD_END reason without creating leases for child shards.");
+ }
+ // Checkpoint with SHARD_END sequence number.
+ final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(currentShardLease, shardInfo);
+ if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) {
+ boolean isSuccess = false;
+ try {
+ isSuccess = attemptShardEndCheckpointing();
+ } 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 void takeLeaseLostAction() {
+ final ShutdownInput leaseLostShutdownInput = new ShutdownInput()
+ .withShutdownReason(ShutdownReason.ZOMBIE)
+ .withCheckpointer(recordProcessorCheckpointer);
+ recordProcessor.shutdown(leaseLostShutdownInput);
+ }
+
+ private boolean attemptShardEndCheckpointing()
+ throws DependencyException, ProvisionedThroughputException, InvalidStateException, CustomerApplicationException {
+ final KinesisClientLease leaseFromDdb = Optional.ofNullable(leaseCoordinator.getLeaseManager().getLease(shardInfo.getShardId()))
+ .orElseThrow(() -> new InvalidStateException("Lease for shard " + shardInfo.getShardId() + " does not exist."));
+ if (!leaseFromDdb.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END)) {
+ // Call the recordProcessor to checkpoint with SHARD_END sequence number.
+ // The recordProcessor.shutdown is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling recordProcessor.shutdown.
+ throwOnApplicationException(() -> applicationCheckpointAndVerification());
+ }
+ return true;
+ }
+
+ private void applicationCheckpointAndVerification() {
+ recordProcessorCheckpointer.setSequenceNumberAtShardEnd(
+ recordProcessorCheckpointer.getLargestPermittedCheckpointValue());
+ recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END);
+ final ShutdownInput shardEndShutdownInput = new ShutdownInput()
+ .withShutdownReason(ShutdownReason.TERMINATE)
+ .withCheckpointer(recordProcessorCheckpointer);
+ recordProcessor.shutdown(shardEndShutdownInput);
+
+ final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.getLastCheckpointValue();
+
+ final boolean successfullyCheckpointedShardEnd = lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END);
+
+ if ((lastCheckpointValue == null) || (!successfullyCheckpointedShardEnd)) {
+ throw new IllegalArgumentException("Application didn't checkpoint at end of shard "
+ + shardInfo.getShardId() + ". Application must checkpoint upon shutdown. " +
+ "See IRecordProcessor.shutdown javadocs for more information.");
+ }
+ }
+
+ private void throwOnApplicationException(Runnable action) throws CustomerApplicationException {
+ try {
+ action.run();
+ } catch (Exception e) {
+ throw new CustomerApplicationException("Customer application throws exception for shard " + shardInfo.getShardId(), e);
+ }
+ }
+
+ private void createLeasesForChildShardsIfNotExist() throws InvalidStateException, DependencyException, 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.getParentShards();
+
+ if (parentLeaseKeys.size() != 2) {
+ throw new InvalidStateException("Shard " + shardInfo.getShardId()+ "'s only child shard " + childShard
+ + " does not contain other parent information.");
+ } else {
+ boolean isValidLeaseTableState = Objects.isNull(leaseCoordinator.getLeaseManager().getLease(parentLeaseKeys.get(0))) ==
+ Objects.isNull(leaseCoordinator.getLeaseManager().getLease(parentLeaseKeys.get(1)));
+ if (!isValidLeaseTableState) {
+ if(!isOneInNProbability(RETRY_RANDOM_MAX_RANGE)) {
+ throw new BlockedOnParentShardException(
+ "Shard " + shardInfo.getShardId() + "'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.getShardId() + "'s only child shard " + childShard
+ + " has partial parent information in lease table.");
+ }
+ }
+ }
+ }
+ // Attempt create leases for child shards.
+ for (ChildShard childShard : childShards) {
+ final String leaseKey = childShard.getShardId();
+ if (leaseCoordinator.getLeaseManager().getLease(leaseKey) == null) {
+ final KinesisClientLease leaseToCreate = KinesisShardSyncer.newKCLLeaseForChildShard(childShard);
+ leaseCoordinator.getLeaseManager().createLeaseIfNotExists(leaseToCreate);
+ LOG.info("Shard " + shardInfo.getShardId() + " : Created child shard lease: " + leaseToCreate.getLeaseKey());
+ }
+ }
+ }
+
+ /**
+ * 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 updateCurrentLeaseWithChildShards(KinesisClientLease currentLease) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+ final Set childShardIds = childShards.stream().map(ChildShard::getShardId).collect(Collectors.toSet());
+ currentLease.setChildShardIds(childShardIds);
+ leaseCoordinator.getLeaseManager().updateLeaseWithMetaInfo(currentLease, UpdateField.CHILD_SHARDS);
+ LOG.info("Shard " + shardInfo.getShardId() + ": Updated current lease with child shard information: " + currentLease.getLeaseKey());
+ }
+
+
/*
* (non-Javadoc)
*
@@ -202,9 +309,12 @@ class ShutdownTask implements ITask {
return reason;
}
- private void dropLease() {
- KinesisClientLease lease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId());
- leaseCoordinator.dropLease(lease);
- LOG.warn("Dropped lease for shutting down ShardConsumer: " + lease.getLeaseKey());
+ private void dropLease(KinesisClientLease currentShardLease) {
+ if (currentShardLease == null) {
+ LOG.warn("Shard " + shardInfo.getShardId() + ": Unable to find the lease for shard. Will shutdown the shardConsumer directly.");
+ return;
+ }
+ leaseCoordinator.dropLease(currentShardLease);
+ LOG.warn("Dropped lease for shutting down ShardConsumer: " + currentShardLease.getLeaseKey());
}
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java
index bc68d292..70109b86 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java
@@ -14,6 +14,10 @@
*/
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+import com.amazonaws.services.kinesis.model.ChildShard;
+
+import java.util.List;
+
/**
* Used to capture information from a task that we want to communicate back to the higher layer.
* E.g. exception thrown when executing the task, if we reach end of a shard.
@@ -26,6 +30,9 @@ class TaskResult {
// Any exception caught while executing the task.
private Exception exception;
+ // List of childShards of the current shard. This field is only required for the task result when we reach end of a shard.
+ private List childShards;
+
/**
* @return the shardEndReached
*/
@@ -33,6 +40,11 @@ class TaskResult {
return shardEndReached;
}
+ /**
+ * @return the list of childShards.
+ */
+ protected List getChildShards() { return childShards; }
+
/**
* @param shardEndReached the shardEndReached to set
*/
@@ -40,6 +52,11 @@ class TaskResult {
this.shardEndReached = shardEndReached;
}
+ /**
+ * @param childShards the list of childShards to set
+ */
+ protected void setChildShards(List childShards) { this.childShards = childShards; }
+
/**
* @return the exception
*/
@@ -70,4 +87,10 @@ class TaskResult {
this.shardEndReached = isShardEndReached;
}
+ TaskResult(Exception e, boolean isShardEndReached, List childShards) {
+ this.exception = e;
+ this.shardEndReached = isShardEndReached;
+ this.childShards = childShards;
+ }
+
}
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java
index 9eabcffe..f8c66181 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java
@@ -29,12 +29,17 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Consumer;
+import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
+import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector;
+import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager;
import com.amazonaws.services.kinesis.leases.impl.LeaseCoordinator;
import com.amazonaws.services.kinesis.leases.impl.LeaseRenewer;
import com.amazonaws.services.kinesis.leases.impl.LeaseTaker;
@@ -88,9 +93,13 @@ public class Worker implements Runnable {
private static final Log LOG = LogFactory.getLog(Worker.class);
+ // Default configs for periodic shard sync
private static final int SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC = 0;
- private static final int MAX_INITIALIZATION_ATTEMPTS = 20;
private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1; //Default for KCL.
+ static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L;
+ static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L;
+ static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L;
+
private static final WorkerStateChangeListener DEFAULT_WORKER_STATE_CHANGE_LISTENER = new NoOpWorkerStateChangeListener();
private static final LeaseCleanupValidator DEFAULT_LEASE_CLEANUP_VALIDATOR = new KinesisLeaseCleanupValidator();
private static final LeaseSelector DEFAULT_LEASE_SELECTOR = new GenericLeaseSelector();
@@ -117,7 +126,7 @@ public class Worker implements Runnable {
private final Optional maxGetRecordsThreadPool;
private final KinesisClientLibLeaseCoordinator leaseCoordinator;
- private final ShardSyncTaskManager controlServer;
+ private final ShardSyncTaskManager shardSyncTaskManager;
private final ShardPrioritization shardPrioritization;
@@ -147,6 +156,9 @@ public class Worker implements Runnable {
// Periodic Shard Sync related fields
private LeaderDecider leaderDecider;
private ShardSyncStrategy shardSyncStrategy;
+ private PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager;
+
+ private final LeaseCleanupManager leaseCleanupManager;
/**
* Constructor.
@@ -406,7 +418,7 @@ public class Worker implements Runnable {
config.getShardPrioritizationStrategy(),
config.getRetryGetRecordsInSeconds(),
config.getMaxGetRecordsThreadPool(),
- DEFAULT_WORKER_STATE_CHANGE_LISTENER, DEFAULT_LEASE_CLEANUP_VALIDATOR, null /* leaderDecider */);
+ DEFAULT_WORKER_STATE_CHANGE_LISTENER, DEFAULT_LEASE_CLEANUP_VALIDATOR, null, null);
// If a region name was explicitly specified, use it as the region for Amazon Kinesis and Amazon DynamoDB.
if (config.getRegionName() != null) {
@@ -467,7 +479,7 @@ public class Worker implements Runnable {
shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService,
metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist,
shardPrioritization, Optional.empty(), Optional.empty(), DEFAULT_WORKER_STATE_CHANGE_LISTENER,
- DEFAULT_LEASE_CLEANUP_VALIDATOR, null);
+ DEFAULT_LEASE_CLEANUP_VALIDATOR, null, null);
}
/**
@@ -507,6 +519,10 @@ public class Worker implements Runnable {
* Max number of threads in the getRecords thread pool.
* @param leaseCleanupValidator
* leaseCleanupValidator instance used to validate leases
+ * @param leaderDecider
+ * leaderDecider instance used elect shard sync leaders
+ * @param periodicShardSyncManager
+ * manages periodic shard sync tasks
*/
// NOTE: This has package level access solely for testing
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
@@ -517,13 +533,13 @@ public class Worker implements Runnable {
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization,
Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool, WorkerStateChangeListener workerStateChangeListener,
- LeaseCleanupValidator leaseCleanupValidator, LeaderDecider leaderDecider) {
+ LeaseCleanupValidator leaseCleanupValidator, LeaderDecider leaderDecider, PeriodicShardSyncManager periodicShardSyncManager) {
this(applicationName, recordProcessorFactory, config, streamConfig, initialPositionInStream,
parentShardPollIntervalMillis, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint,
leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis,
skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization, retryGetRecordsInSeconds,
maxGetRecordsThreadPool, workerStateChangeListener, new KinesisShardSyncer(leaseCleanupValidator),
- leaderDecider);
+ leaderDecider, periodicShardSyncManager);
}
Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config,
@@ -533,7 +549,8 @@ public class Worker implements Runnable {
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization,
Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool,
- WorkerStateChangeListener workerStateChangeListener, ShardSyncer shardSyncer, LeaderDecider leaderDecider) {
+ WorkerStateChangeListener workerStateChangeListener, ShardSyncer shardSyncer, LeaderDecider leaderDecider,
+ PeriodicShardSyncManager periodicShardSyncManager) {
this.applicationName = applicationName;
this.recordProcessorFactory = recordProcessorFactory;
this.config = config;
@@ -547,7 +564,7 @@ public class Worker implements Runnable {
this.leaseCoordinator = leaseCoordinator;
this.metricsFactory = metricsFactory;
this.shardSyncer = shardSyncer;
- this.controlServer = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(),
+ this.shardSyncTaskManager = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(),
initialPositionInStream, cleanupLeasesUponShardCompletion, config.shouldIgnoreUnexpectedChildShards(),
shardSyncIdleTimeMillis, metricsFactory, executorService, shardSyncer);
this.taskBackoffTimeMillis = taskBackoffTimeMillis;
@@ -558,19 +575,42 @@ public class Worker implements Runnable {
this.maxGetRecordsThreadPool = maxGetRecordsThreadPool;
this.workerStateChangeListener = workerStateChangeListener;
workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.CREATED);
- this.leaderDecider = leaderDecider;
- this.shardSyncStrategy = createShardSyncStrategy(config.getShardSyncStrategyType());
- LOG.info(String.format("Shard sync strategy determined as %s.", shardSyncStrategy.getStrategyType().toString()));
+ createShardSyncStrategy(config.getShardSyncStrategyType(), leaderDecider, periodicShardSyncManager);
+ this.leaseCleanupManager = LeaseCleanupManager.createOrGetInstance(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(),
+ Executors.newSingleThreadScheduledExecutor(), metricsFactory, cleanupLeasesUponShardCompletion,
+ config.leaseCleanupIntervalMillis(), config.completedLeaseCleanupThresholdMillis(),
+ config.garbageLeaseCleanupThresholdMillis(), config.getMaxRecords());
}
- private ShardSyncStrategy createShardSyncStrategy(ShardSyncStrategyType strategyType) {
+ /**
+ * Create shard sync strategy and corresponding {@link LeaderDecider} based on provided configs. PERIODIC
+ * {@link ShardSyncStrategyType} honors custom leaderDeciders for leader election strategy, and does not permit
+ * skipping shard syncs if the hash range is complete. All other {@link ShardSyncStrategyType}s permit only a
+ * default single-leader strategy, and will skip shard syncs unless a hole in the hash range is detected.
+ */
+ private void createShardSyncStrategy(ShardSyncStrategyType strategyType,
+ LeaderDecider leaderDecider,
+ PeriodicShardSyncManager periodicShardSyncManager) {
switch (strategyType) {
case PERIODIC:
- return createPeriodicShardSyncStrategy(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager());
+ this.leaderDecider = getOrCreateLeaderDecider(leaderDecider);
+ this.leaderElectedPeriodicShardSyncManager =
+ getOrCreatePeriodicShardSyncManager(periodicShardSyncManager, false);
+ this.shardSyncStrategy = createPeriodicShardSyncStrategy();
+ break;
case SHARD_END:
default:
- return createShardEndShardSyncStrategy(controlServer);
+ if (leaderDecider != null) {
+ LOG.warn("LeaderDecider cannot be customized with non-PERIODIC shard sync strategy type. Using " +
+ "default LeaderDecider.");
+ }
+ this.leaderDecider = getOrCreateLeaderDecider(null);
+ this.leaderElectedPeriodicShardSyncManager =
+ getOrCreatePeriodicShardSyncManager(periodicShardSyncManager, true);
+ this.shardSyncStrategy = createShardEndShardSyncStrategy();
}
+
+ LOG.info("Shard sync strategy determined as " + shardSyncStrategy.getStrategyType().toString());
}
private static KinesisClientLibLeaseCoordinator getLeaseCoordinator(KinesisClientLibConfiguration config,
@@ -602,6 +642,20 @@ public class Worker implements Runnable {
return leaseCoordinator;
}
+ /**
+ * @return the leaderDecider
+ */
+ LeaderDecider getLeaderDecider() {
+ return leaderDecider;
+ }
+
+ /**
+ * @return the leaderElectedPeriodicShardSyncManager
+ */
+ PeriodicShardSyncManager getPeriodicShardSyncManager() {
+ return leaderElectedPeriodicShardSyncManager;
+ }
+
/**
* Start consuming data from the stream, and pass it to the application record processors.
*/
@@ -614,7 +668,8 @@ public class Worker implements Runnable {
initialize();
LOG.info("Initialization complete. Starting worker loop.");
} catch (RuntimeException e1) {
- LOG.error("Unable to initialize after " + MAX_INITIALIZATION_ATTEMPTS + " attempts. Shutting down.", e1);
+ LOG.error("Unable to initialize after " + config.getMaxInitializationAttempts() + " attempts. " +
+ "Shutting down.", e1);
shutdown();
}
@@ -641,10 +696,6 @@ public class Worker implements Runnable {
assignedShards.add(shardInfo);
}
- if (foundCompletedShard) {
- shardSyncStrategy.onFoundCompletedShard();
- }
-
// clean up shard consumers for unassigned shards
cleanupShardConsumers(assignedShards);
@@ -667,36 +718,38 @@ public class Worker implements Runnable {
boolean isDone = false;
Exception lastException = null;
- for (int i = 0; (!isDone) && (i < MAX_INITIALIZATION_ATTEMPTS); i++) {
+ for (int i = 0; (!isDone) && (i < config.getMaxInitializationAttempts()); i++) {
try {
LOG.info("Initialization attempt " + (i + 1));
LOG.info("Initializing LeaseCoordinator");
leaseCoordinator.initialize();
- TaskResult result = null;
- if (!skipShardSyncAtWorkerInitializationIfLeasesExist
- || leaseCoordinator.getLeaseManager().isLeaseTableEmpty()) {
- LOG.info("Syncing Kinesis shard info");
- ShardSyncTask shardSyncTask = new ShardSyncTask(streamConfig.getStreamProxy(),
- leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion,
- config.shouldIgnoreUnexpectedChildShards(), 0L, shardSyncer, null);
- result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call();
- } else {
- LOG.info("Skipping shard sync per config setting (and lease table is not empty)");
+ // Perform initial lease sync if configs allow it, with jitter.
+ if (shouldInitiateLeaseSync()) {
+ LOG.info(config.getWorkerIdentifier() + " worker is beginning initial lease sync.");
+ TaskResult result = leaderElectedPeriodicShardSyncManager.syncShardsOnce();
+ if (result.getException() != null) {
+ throw result.getException();
+ }
}
- 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.");
- }
- shardSyncStrategy.onWorkerInitialization();
- isDone = true;
+ leaseCleanupManager.start();
+
+ // If we reach this point, then we either skipped the lease sync or did not have any exception for 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.");
}
+
+ // All shard sync strategies' initialization handlers should begin a periodic shard sync. For
+ // PeriodicShardSync strategy, this is the main shard sync loop. For ShardEndShardSync and other
+ // shard sync strategies, this serves as an auditor background process.
+ shardSyncStrategy.onWorkerInitialization();
+ isDone = true;
+
} catch (LeasingException e) {
LOG.error("Caught exception when initializing LeaseCoordinator", e);
lastException = e;
@@ -712,11 +765,39 @@ public class Worker implements Runnable {
}
if (!isDone) {
+ leaderElectedPeriodicShardSyncManager.stop();
throw new RuntimeException(lastException);
+
}
workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.STARTED);
}
+ @VisibleForTesting
+ boolean shouldInitiateLeaseSync() throws InterruptedException, DependencyException, InvalidStateException,
+ ProvisionedThroughputException {
+
+ final ILeaseManager leaseManager = leaseCoordinator.getLeaseManager();
+ if (skipShardSyncAtWorkerInitializationIfLeasesExist && !leaseManager.isLeaseTableEmpty()) {
+ LOG.info("Skipping shard sync because getSkipShardSyncAtWorkerInitializationIfLeasesExist config is set " +
+ "to TRUE and lease table is not empty.");
+ return false;
+ }
+
+ final long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS,
+ MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS);
+ final long waitUntil = System.currentTimeMillis() + waitTime;
+
+ boolean shouldInitiateLeaseSync = true;
+ while (System.currentTimeMillis() < waitUntil && (shouldInitiateLeaseSync = leaseManager.isLeaseTableEmpty())) {
+ // Check every 3 seconds if lease table is still empty, to minimize contention between all workers
+ // bootstrapping from empty lease table at the same time.
+ LOG.info("Lease table is still empty. Checking again in " + LEASE_TABLE_CHECK_FREQUENCY_MILLIS + " ms.");
+ Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS);
+ }
+
+ return shouldInitiateLeaseSync;
+ }
+
/**
* NOTE: This method is internal/private to the Worker class. It has package access solely for testing.
*
@@ -1039,12 +1120,21 @@ public class Worker implements Runnable {
}
protected ShardConsumer buildConsumer(ShardInfo shardInfo, IRecordProcessorFactory processorFactory) {
- IRecordProcessor recordProcessor = processorFactory.createProcessor();
+ final IRecordProcessor recordProcessor = processorFactory.createProcessor();
+ final RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer(
+ shardInfo,
+ checkpointTracker,
+ new SequenceNumberValidator(
+ streamConfig.getStreamProxy(),
+ shardInfo.getShardId(),
+ streamConfig.shouldValidateSequenceNumberBeforeCheckpointing()),
+ metricsFactory);
return new ShardConsumer(shardInfo,
streamConfig,
checkpointTracker,
recordProcessor,
+ recordProcessorCheckpointer,
leaseCoordinator,
parentShardPollIntervalMillis,
cleanupLeasesUponShardCompletion,
@@ -1052,9 +1142,11 @@ public class Worker implements Runnable {
metricsFactory,
taskBackoffTimeMillis,
skipShardSyncAtWorkerInitializationIfLeasesExist,
+ new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo),
retryGetRecordsInSeconds,
maxGetRecordsThreadPool,
- config, shardSyncer, shardSyncStrategy);
+ config, shardSyncer, shardSyncStrategy,
+ leaseCleanupManager);
}
/**
@@ -1163,18 +1255,47 @@ public class Worker implements Runnable {
}
}
- private PeriodicShardSyncStrategy createPeriodicShardSyncStrategy(IKinesisProxy kinesisProxy,
- ILeaseManager leaseManager) {
- return new PeriodicShardSyncStrategy(
- new PeriodicShardSyncManager(config.getWorkerIdentifier(), leaderDecider,
- new ShardSyncTask(kinesisProxy, leaseManager, config.getInitialPositionInStreamExtended(),
- config.shouldCleanupLeasesUponShardCompletion(),
- config.shouldIgnoreUnexpectedChildShards(), SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC,
- shardSyncer, null), metricsFactory));
+ private PeriodicShardSyncStrategy createPeriodicShardSyncStrategy() {
+ return new PeriodicShardSyncStrategy(leaderElectedPeriodicShardSyncManager);
}
- private ShardEndShardSyncStrategy createShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager) {
- return new ShardEndShardSyncStrategy(shardSyncTaskManager);
+ private ShardEndShardSyncStrategy createShardEndShardSyncStrategy() {
+ return new ShardEndShardSyncStrategy(shardSyncTaskManager, leaderElectedPeriodicShardSyncManager);
+ }
+
+ private LeaderDecider getOrCreateLeaderDecider(LeaderDecider leaderDecider) {
+ if (leaderDecider != null) {
+ return leaderDecider;
+ }
+
+ return new DeterministicShuffleShardSyncLeaderDecider(leaseCoordinator.getLeaseManager(),
+ Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT);
+ }
+
+ /** A non-null PeriodicShardSyncManager can only provided from unit tests. Any application code will create the
+ * PeriodicShardSyncManager for the first time here. */
+ private PeriodicShardSyncManager getOrCreatePeriodicShardSyncManager(PeriodicShardSyncManager periodicShardSyncManager,
+ boolean isAuditorMode) {
+ if (periodicShardSyncManager != null) {
+ return periodicShardSyncManager;
+ }
+
+ return new PeriodicShardSyncManager(config.getWorkerIdentifier(),
+ leaderDecider,
+ new ShardSyncTask(streamConfig.getStreamProxy(),
+ leaseCoordinator.getLeaseManager(),
+ config.getInitialPositionInStreamExtended(),
+ config.shouldCleanupLeasesUponShardCompletion(),
+ config.shouldIgnoreUnexpectedChildShards(),
+ SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC,
+ shardSyncer,
+ null),
+ metricsFactory,
+ leaseCoordinator.getLeaseManager(),
+ streamConfig.getStreamProxy(),
+ isAuditorMode,
+ config.getLeasesRecoveryAuditorExecutionFrequencyMillis(),
+ config.getLeasesRecoveryAuditorInconsistencyConfidenceThreshold());
}
/**
@@ -1242,7 +1363,6 @@ public class Worker implements Runnable {
@Setter @Accessors(fluent = true)
private ShardSyncer shardSyncer;
-
@VisibleForTesting
AmazonKinesis getKinesisClient() {
return kinesisClient;
@@ -1349,7 +1469,7 @@ public class Worker implements Runnable {
}
if (shardPrioritization == null) {
- shardPrioritization = new ParentsFirstShardPrioritization(1);
+ shardPrioritization = new NoOpShardPrioritization();
}
if (kinesisProxy == null) {
@@ -1379,7 +1499,7 @@ public class Worker implements Runnable {
}
// We expect users to either inject both LeaseRenewer and the corresponding thread-pool, or neither of them (DEFAULT).
- if (leaseRenewer == null){
+ if (leaseRenewer == null) {
ExecutorService leaseRenewerThreadPool = LeaseCoordinator.getDefaultLeaseRenewalExecutorService(config.getMaxLeaseRenewalThreads());
leaseRenewer = new LeaseRenewer<>(leaseManager, config.getWorkerIdentifier(), config.getFailoverTimeMillis(), leaseRenewerThreadPool);
}
@@ -1419,7 +1539,10 @@ public class Worker implements Runnable {
shardPrioritization,
config.getRetryGetRecordsInSeconds(),
config.getMaxGetRecordsThreadPool(),
- workerStateChangeListener, shardSyncer, leaderDecider);
+ workerStateChangeListener,
+ shardSyncer,
+ leaderDecider,
+ null /* PeriodicShardSyncManager */);
}
> R createClient(final T builder,
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java
index 6e148969..7921a321 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java
@@ -26,6 +26,7 @@ import com.amazonaws.services.kinesis.model.InvalidArgumentException;
import com.amazonaws.services.kinesis.model.PutRecordResult;
import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.services.kinesis.model.ShardFilter;
/**
* Kinesis proxy interface. Operates on a single stream (set up at initialization).
@@ -78,6 +79,17 @@ public interface IKinesisProxy {
*/
List getShardList() throws ResourceNotFoundException;
+ /**
+ * Fetch a subset shards defined for the stream using a filter on the ListShards API. This can be used to
+ * discover new shards and consume data from them, while limiting the total number of shards returned for
+ * performance or efficiency reasons.
+ *
+ * @param shardFilter currently supported filter types are AT_LATEST, AT_TRIM_HORIZON, AT_TIMESTAMP.
+ * @return List of all shards in the Kinesis stream.
+ * @throws ResourceNotFoundException The Kinesis stream was not found.
+ */
+ List getShardListWithFilter(ShardFilter shardFilter) throws ResourceNotFoundException;
+
/**
* Used to verify during ShardConsumer shutdown if the provided shardId is for a shard that has been closed.
* @param shardId Id of the shard that needs to be verified.
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java
index 6717208b..0936bddb 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java
@@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Function;
import java.util.stream.Collectors;
+import com.amazonaws.services.kinesis.model.ShardFilter;
import com.amazonaws.util.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
@@ -309,7 +310,7 @@ public class KinesisProxy implements IKinesisProxyExtended {
}
}
- private ListShardsResult listShards(final String nextToken) {
+ private ListShardsResult listShards(final ShardFilter shardFilter, final String nextToken) {
final ListShardsRequest request = new ListShardsRequest();
request.setRequestCredentials(credentialsProvider.getCredentials());
if (StringUtils.isEmpty(nextToken)) {
@@ -317,6 +318,11 @@ public class KinesisProxy implements IKinesisProxyExtended {
} else {
request.setNextToken(nextToken);
}
+
+ if (shardFilter != null) {
+ request.setShardFilter(shardFilter);
+ }
+
ListShardsResult result = null;
LimitExceededException lastException = null;
int remainingRetries = this.maxListShardsRetryAttempts;
@@ -429,29 +435,37 @@ public class KinesisProxy implements IKinesisProxyExtended {
*/
@Override
public synchronized List getShardList() {
+ return getShardListWithFilter(null);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public synchronized List getShardListWithFilter(ShardFilter shardFilter) {
if (shardIterationState == null) {
shardIterationState = new ShardIterationState();
}
-
+
if (isKinesisClient) {
ListShardsResult result;
String nextToken = null;
-
+
do {
- result = listShards(nextToken);
-
+ result = listShards(shardFilter, nextToken);
+
if (result == null) {
/*
- * If listShards ever returns null, we should bail and return null. This indicates the stream is not
- * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream.
- */
+ * If listShards ever returns null, we should bail and return null. This indicates the stream is not
+ * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream.
+ */
return null;
} else {
shardIterationState.update(result.getShards());
nextToken = result.getNextToken();
}
} while (StringUtils.isNotEmpty(result.getNextToken()));
-
+
} else {
DescribeStreamResult response;
@@ -459,10 +473,10 @@ public class KinesisProxy implements IKinesisProxyExtended {
response = getStreamInfo(shardIterationState.getLastShardId());
if (response == null) {
- /*
- * If getStreamInfo ever returns null, we should bail and return null. This indicates the stream is not
- * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream.
- */
+ /*
+ * If getStreamInfo ever returns null, we should bail and return null. This indicates the stream is not
+ * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream.
+ */
return null;
} else {
shardIterationState.update(response.getStreamDescription().getShards());
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java
index 230ee710..fe6eb51e 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java
@@ -28,6 +28,7 @@ import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
import com.amazonaws.services.kinesis.model.Shard;
import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper;
import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel;
+import com.amazonaws.services.kinesis.model.ShardFilter;
/**
* IKinesisProxy implementation that wraps another implementation and collects metrics.
@@ -179,6 +180,22 @@ public class MetricsCollectingKinesisProxyDecorator implements IKinesisProxy {
}
}
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public List getShardListWithFilter(ShardFilter shardFilter) throws ResourceNotFoundException {
+ long startTime = System.currentTimeMillis();
+ boolean success = false;
+ try {
+ List response = other.getShardListWithFilter(shardFilter);
+ success = true;
+ return response;
+ } finally {
+ MetricsHelper.addSuccessAndLatency(getShardListMetric, startTime, success, MetricsLevel.DETAILED);
+ }
+ }
+
/**
* {@inheritDoc}
*/
diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java
index e817e0ea..92d1e71a 100644
--- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java
+++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java
@@ -141,6 +141,10 @@ public class ExtendedSequenceNumber implements Comparable parentShardIds = new HashSet();
+ private Set childShardIds = new HashSet<>();
+ private HashKeyRangeForLease hashKeyRangeForLease;
+
public KinesisClientLease() {
@@ -41,17 +44,22 @@ public class KinesisClientLease extends Lease {
this.pendingCheckpoint = other.getPendingCheckpoint();
this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint();
this.parentShardIds.addAll(other.getParentShardIds());
+ this.childShardIds.addAll(other.getChildShardIds());
+ this.hashKeyRangeForLease = other.getHashKeyRange();
}
KinesisClientLease(String leaseKey, String leaseOwner, Long leaseCounter, UUID concurrencyToken,
Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint,
- Long ownerSwitchesSinceCheckpoint, Set parentShardIds) {
+ Long ownerSwitchesSinceCheckpoint, Set parentShardIds, Set childShardIds,
+ HashKeyRangeForLease hashKeyRangeForLease) {
super(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos);
this.checkpoint = checkpoint;
this.pendingCheckpoint = pendingCheckpoint;
this.ownerSwitchesSinceCheckpoint = ownerSwitchesSinceCheckpoint;
this.parentShardIds.addAll(parentShardIds);
+ this.childShardIds.addAll(childShardIds);
+ this.hashKeyRangeForLease = hashKeyRangeForLease;
}
/**
@@ -69,6 +77,7 @@ public class KinesisClientLease extends Lease {
setCheckpoint(casted.checkpoint);
setPendingCheckpoint(casted.pendingCheckpoint);
setParentShardIds(casted.parentShardIds);
+ setChildShardIds(casted.childShardIds);
}
/**
@@ -100,6 +109,20 @@ public class KinesisClientLease extends Lease {
return new HashSet(parentShardIds);
}
+ /**
+ * @return shardIds for the child shards of the current shard. Used for resharding.
+ */
+ public Set getChildShardIds() {
+ return new HashSet(childShardIds);
+ }
+
+ /**
+ * @return hash key range that this lease covers.
+ */
+ public HashKeyRangeForLease getHashKeyRange() {
+ return hashKeyRangeForLease;
+ }
+
/**
* Sets checkpoint.
*
@@ -142,7 +165,27 @@ public class KinesisClientLease extends Lease {
this.parentShardIds.clear();
this.parentShardIds.addAll(parentShardIds);
}
-
+
+ /**
+ * Sets childShardIds.
+ *
+ * @param childShardIds may not be null
+ */
+ public void setChildShardIds(Collection childShardIds) {
+ this.childShardIds.addAll(childShardIds);
+ }
+
+ /**
+ * Sets hashKeyRangeForLease.
+ *
+ * @param hashKeyRangeForLease may not be null
+ */
+ public void setHashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) {
+ verifyNotNull(hashKeyRangeForLease, "hashKeyRangeForLease should not be null");
+
+ this.hashKeyRangeForLease = hashKeyRangeForLease;
+ }
+
private void verifyNotNull(Object object, String message) {
if (object == null) {
throw new IllegalArgumentException(message);
diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java
index 4006e052..310edb67 100644
--- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java
@@ -15,6 +15,7 @@
package com.amazonaws.services.kinesis.leases.impl;
import java.util.Collection;
+import java.util.HashMap;
import java.util.Map;
import com.amazonaws.services.dynamodbv2.model.AttributeAction;
@@ -26,8 +27,11 @@ import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
import com.amazonaws.services.kinesis.leases.interfaces.ILeaseSerializer;
import com.amazonaws.services.kinesis.leases.util.DynamoUtils;
+import com.amazonaws.util.CollectionUtils;
import com.google.common.base.Strings;
+import static com.amazonaws.services.kinesis.leases.impl.UpdateField.HASH_KEY_RANGE;
+
/**
* An implementation of ILeaseSerializer for KinesisClientLease objects.
*/
@@ -39,6 +43,9 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer getDynamoExistentExpectation(final String leaseKey) {
+ return baseSerializer.getDynamoExistentExpectation(leaseKey);
+ }
+
@Override
public Map getDynamoLeaseCounterUpdate(KinesisClientLease lease) {
return baseSerializer.getDynamoLeaseCounterUpdate(lease);
@@ -143,6 +159,9 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer getDynamoUpdateLeaseUpdate(KinesisClientLease lease,
+ UpdateField updateField) {
+ Map result = new HashMap<>();
+
+ switch (updateField) {
+ case CHILD_SHARDS:
+ // TODO: Implement update fields for child shards
+ break;
+ case HASH_KEY_RANGE:
+ if (lease.getHashKeyRange() != null) {
+ result.put(STARTING_HASH_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue(
+ lease.getHashKeyRange().serializedStartingHashKey()), AttributeAction.PUT));
+ result.put(ENDING_HASH_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue(
+ lease.getHashKeyRange().serializedEndingHashKey()), AttributeAction.PUT));
+ }
+ break;
+ }
+
+ return result;
+ }
+
@Override
public Collection getKeySchema() {
return baseSerializer.getKeySchema();
diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java
new file mode 100644
index 00000000..b3157e78
--- /dev/null
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java
@@ -0,0 +1,372 @@
+package com.amazonaws.services.kinesis.leases.impl;
+
+/*
+ * 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.
+ */
+
+import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShardInfo;
+import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy;
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.leases.LeasePendingDeletion;
+import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
+import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
+import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
+import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
+import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
+import com.amazonaws.services.kinesis.model.ResourceNotFoundException;
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+import com.amazonaws.util.CollectionUtils;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import lombok.AccessLevel;
+import lombok.EqualsAndHashCode;
+import lombok.Getter;
+import lombok.NonNull;
+import lombok.RequiredArgsConstructor;
+import lombok.Value;
+import lombok.experimental.Accessors;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+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.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+/**
+ * Helper class to cleanup of any expired/closed shard leases. It will cleanup leases periodically as defined by
+ * {@link KinesisClientLibConfiguration#leaseCleanupIntervalMillis()} upon worker shutdown, following a re-shard event or
+ * a shard expiring from the service.
+ */
+@RequiredArgsConstructor(access= AccessLevel.PACKAGE)
+@EqualsAndHashCode
+public class LeaseCleanupManager {
+ @NonNull
+ private IKinesisProxy kinesisProxy;
+ @NonNull
+ private final ILeaseManager leaseManager;
+ @NonNull
+ private final ScheduledExecutorService deletionThreadPool;
+ @NonNull
+ private final IMetricsFactory metricsFactory;
+ private final boolean cleanupLeasesUponShardCompletion;
+ private final long leaseCleanupIntervalMillis;
+ private final long completedLeaseCleanupIntervalMillis;
+ private final long garbageLeaseCleanupIntervalMillis;
+ private final int maxRecords;
+
+ 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;
+ private static final Log LOG = LogFactory.getLog(LeaseCleanupManager.class);
+
+ @Getter
+ private volatile boolean isRunning = false;
+
+ private static LeaseCleanupManager instance;
+
+ /**
+ * Factory method to return a singleton instance of {@link LeaseCleanupManager}.
+ * @param kinesisProxy
+ * @param leaseManager
+ * @param deletionThreadPool
+ * @param metricsFactory
+ * @param cleanupLeasesUponShardCompletion
+ * @param leaseCleanupIntervalMillis
+ * @param completedLeaseCleanupIntervalMillis
+ * @param garbageLeaseCleanupIntervalMillis
+ * @param maxRecords
+ * @return
+ */
+ public static LeaseCleanupManager createOrGetInstance(IKinesisProxy kinesisProxy, ILeaseManager leaseManager,
+ ScheduledExecutorService deletionThreadPool, IMetricsFactory metricsFactory,
+ boolean cleanupLeasesUponShardCompletion, long leaseCleanupIntervalMillis,
+ long completedLeaseCleanupIntervalMillis, long garbageLeaseCleanupIntervalMillis,
+ int maxRecords) {
+ if (instance == null) {
+ instance = new LeaseCleanupManager(kinesisProxy, leaseManager, deletionThreadPool, metricsFactory, cleanupLeasesUponShardCompletion,
+ leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis, maxRecords);
+ }
+
+ return instance;
+ }
+
+ /**
+ * 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.start();
+ garbageLeaseStopwatch.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 KinesisClientLease lease = leasePendingDeletion.lease();
+ if (lease == null) {
+ LOG.warn("Cannot enqueue lease " + lease.getLeaseKey() + " for deferred deletion - instance doesn't hold " +
+ "the lease for that shard.");
+ } else {
+ LOG.debug("Enqueuing lease " + lease.getLeaseKey() + " for deferred deletion.");
+ if (!deletionQueue.add(leasePendingDeletion)) {
+ LOG.warn("Unable to enqueue lease " + lease.getLeaseKey() + " for deletion.");
+ }
+ }
+ }
+
+ /**
+ * 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 DependencyException, ProvisionedThroughputException, InvalidStateException {
+ final KinesisClientLease lease = leasePendingDeletion.lease();
+ final ShardInfo shardInfo = leasePendingDeletion.shardInfo();
+
+ boolean cleanedUpCompletedLease = false;
+ boolean cleanedUpGarbageLease = false;
+ boolean alreadyCheckedForGarbageCollection = false;
+ boolean wereChildShardsPresent = false;
+ boolean wasResourceNotFound = false;
+
+ try {
+ if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) {
+ final KinesisClientLease leaseFromDDB = leaseManager.getLease(shardInfo.getShardId());
+ if(leaseFromDDB != null) {
+ Set childShardKeys = leaseFromDDB.getChildShardIds();
+ if (CollectionUtils.isNullOrEmpty(childShardKeys)) {
+ try {
+ childShardKeys = getChildShardsFromService(shardInfo);
+
+ if (CollectionUtils.isNullOrEmpty(childShardKeys)) {
+ LOG.error("No child shards returned from service for shard " + shardInfo.getShardId());
+ } else {
+ wereChildShardsPresent = true;
+ updateLeaseWithChildShards(leasePendingDeletion, childShardKeys);
+ }
+ } catch (ResourceNotFoundException e) {
+ throw e;
+ } finally {
+ alreadyCheckedForGarbageCollection = true;
+ }
+ } else {
+ wereChildShardsPresent = true;
+ }
+ try {
+ cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, childShardKeys);
+ } catch (Exception e) {
+ // Suppressing the exception here, so that we can attempt for garbage cleanup.
+ LOG.warn("Unable to cleanup lease for shard " + shardInfo.getShardId());
+ }
+ } else {
+ LOG.info("Lease not present in lease table while cleaning the shard " + shardInfo.getShardId());
+ cleanedUpCompletedLease = true;
+ }
+ }
+
+ if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) {
+ try {
+ wereChildShardsPresent = !CollectionUtils
+ .isNullOrEmpty(getChildShardsFromService(shardInfo));
+ } catch (ResourceNotFoundException e) {
+ throw e;
+ }
+ }
+ } catch (ResourceNotFoundException e) {
+ wasResourceNotFound = true;
+ cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease);
+ }
+
+ return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease, wereChildShardsPresent,
+ wasResourceNotFound);
+ }
+
+ private Set getChildShardsFromService(ShardInfo shardInfo) {
+ final String iterator = kinesisProxy.getIterator(shardInfo.getShardId(), ShardIteratorType.LATEST.toString());
+ return kinesisProxy.get(iterator, maxRecords).getChildShards().stream().map(c -> c.getShardId()).collect(Collectors.toSet());
+ }
+
+
+ // 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(KinesisClientLease lease) throws DependencyException, ProvisionedThroughputException, InvalidStateException {
+ LOG.info("Deleting lease " + lease.getLeaseKey() + " as it is not present in the stream.");
+ leaseManager.deleteLease(lease);
+ return true;
+ }
+
+ private boolean allParentShardLeasesDeleted(KinesisClientLease lease) throws DependencyException, ProvisionedThroughputException, InvalidStateException {
+ for (String parentShard : lease.getParentShardIds()) {
+ final KinesisClientLease parentLease = leaseManager.getLease(parentShard);
+
+ if (parentLease != null) {
+ LOG.warn("Lease " + lease.getLeaseKey() + " has a parent lease " + parentLease.getLeaseKey() +
+ " which is still present in the lease table, skipping deletion for this lease.");
+ 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(KinesisClientLease lease, Set childShardLeaseKeys)
+ throws DependencyException, ProvisionedThroughputException, InvalidStateException, IllegalStateException {
+ final Set processedChildShardLeaseKeys = new HashSet<>();
+
+ for (String childShardLeaseKey : childShardLeaseKeys) {
+ final KinesisClientLease childShardLease = Optional.ofNullable(
+ leaseManager.getLease(childShardLeaseKey))
+ .orElseThrow(() -> new IllegalStateException(
+ "Child lease " + childShardLeaseKey + " for completed shard not found in "
+ + "lease table - not cleaning up lease " + lease));
+
+ if (!childShardLease.getCheckpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) && !childShardLease
+ .getCheckpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) {
+ processedChildShardLeaseKeys.add(childShardLease.getLeaseKey());
+ }
+ }
+
+ if (!allParentShardLeasesDeleted(lease) || !Objects.equals(childShardLeaseKeys, processedChildShardLeaseKeys)) {
+ return false;
+ }
+
+ LOG.info("Deleting lease " + lease.getLeaseKey() + " as it has been completely processed and processing of child shard(s) has begun.");
+ leaseManager.deleteLease(lease);
+
+ return true;
+ }
+
+ private void updateLeaseWithChildShards(LeasePendingDeletion leasePendingDeletion, Set childShardKeys)
+ throws DependencyException, ProvisionedThroughputException, InvalidStateException {
+ final KinesisClientLease updatedLease = leasePendingDeletion.lease();
+ updatedLease.setChildShardIds(childShardKeys);
+
+ leaseManager.updateLease(updatedLease);
+ }
+
+ @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 " + deletionQueue.size() + " lease(s).");
+
+ while (!deletionQueue.isEmpty()) {
+ final LeasePendingDeletion leasePendingDeletion = deletionQueue.poll();
+ final String leaseKey = leasePendingDeletion.lease().getLeaseKey();
+ boolean deletionSucceeded = false;
+ try {
+ final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion,
+ timeToCheckForCompletedShard(), timeToCheckForGarbageShard());
+ completedLeaseCleanedUp |= leaseCleanupResult.cleanedUpCompletedLease();
+ garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease();
+
+ if (leaseCleanupResult.leaseCleanedUp()) {
+ LOG.debug("Successfully cleaned up lease " + leaseKey);
+ deletionSucceeded = true;
+ } else {
+ LOG.warn("Unable to clean up lease " + leaseKey + " due to " + leaseCleanupResult);
+ }
+ } catch (Exception e) {
+ LOG.error("Failed to cleanup lease " + leaseKey + ". Will re-enqueue for deletion and retry on next " +
+ "scheduled execution.", e);
+ }
+ if (!deletionSucceeded) {
+ LOG.debug("Did not cleanup lease " + leaseKey + ". Re-enqueueing for deletion.");
+ 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
+ @Accessors(fluent=true)
+ public static class LeaseCleanupResult {
+ boolean cleanedUpCompletedLease;
+ boolean cleanedUpGarbageLease;
+ boolean wereChildShardsPresent;
+ boolean wasResourceNotFound;
+
+ public boolean leaseCleanedUp() {
+ return cleanedUpCompletedLease | cleanedUpGarbageLease;
+ }
+ }
+}
diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java
index 48c72e88..00a3e755 100644
--- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java
@@ -241,7 +241,7 @@ public class LeaseManager implements ILeaseManager {
*/
@Override
public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
- return list(1).isEmpty();
+ return list(1, 1).isEmpty();
}
/**
@@ -254,6 +254,20 @@ public class LeaseManager implements ILeaseManager {
* @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity
*/
List list(Integer limit) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+ return list(limit, Integer.MAX_VALUE);
+ }
+
+ /**
+ * List with the given page size, up to a limit of paginated calls.
+ *
+ * @param limit number of items to consider at a time - used by integration tests to force paging.
+ * @param maxPages max number of paginated scan calls.
+ * @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) throws InvalidStateException, ProvisionedThroughputException, DependencyException {
if (LOG.isDebugEnabled()) {
LOG.debug("Listing leases from table " + table);
}
@@ -278,7 +292,7 @@ public class LeaseManager implements ILeaseManager {
}
Map lastEvaluatedKey = scanResult.getLastEvaluatedKey();
- if (lastEvaluatedKey == null) {
+ if (lastEvaluatedKey == null || --maxPages <= 0) {
// Signify that we're done.
scanResult = null;
if (LOG.isDebugEnabled()) {
@@ -591,6 +605,37 @@ public class LeaseManager implements ILeaseManager {
return true;
}
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void updateLeaseWithMetaInfo(T lease, UpdateField updateField)
+ throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+ verifyNotNull(lease, "lease cannot be null");
+ verifyNotNull(updateField, "updateField cannot be null");
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Updating lease " + lease + " for field " + updateField);
+ }
+
+ UpdateItemRequest request = new UpdateItemRequest();
+ request.setTableName(table);
+ request.setKey(serializer.getDynamoHashKey(lease));
+ request.setExpected(serializer.getDynamoExistentExpectation(lease.getLeaseKey()));
+
+ Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField);
+ updates.putAll(serializer.getDynamoUpdateLeaseUpdate(lease));
+ request.setAttributeUpdates(updates);
+
+ try {
+ dynamoDBClient.updateItem(request);
+ } catch (ConditionalCheckFailedException e) {
+ LOG.warn("Lease update failed for lease with key " + lease.getLeaseKey() + " because the lease did not exist at the time of the update", e);
+ } catch (AmazonClientException e) {
+ throw convertAndRethrowExceptions("update", lease.getLeaseKey(), e);
+ }
+ }
+
/*
* This method contains boilerplate exception handling - it throws or returns something to be thrown. The
* inconsistency there exists to satisfy the compiler when this method is used at the end of non-void methods.
diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java
index 46c45c24..85381560 100644
--- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java
@@ -137,6 +137,18 @@ public class LeaseSerializer implements ILeaseSerializer {
return result;
}
+ @Override
+ public Map getDynamoExistentExpectation(final String leaseKey) {
+ Map result = new HashMap<>();
+
+ ExpectedAttributeValue expectedAV = new ExpectedAttributeValue();
+ expectedAV.setValue(DynamoUtils.createAttributeValue(leaseKey));
+ expectedAV.setExists(true);
+ result.put(LEASE_KEY_KEY, expectedAV);
+
+ return result;
+ }
+
@Override
public Map getDynamoLeaseCounterUpdate(Lease lease) {
return getDynamoLeaseCounterUpdate(lease.getLeaseCounter());
@@ -177,6 +189,12 @@ public class LeaseSerializer implements ILeaseSerializer {
return new HashMap();
}
+ @Override
+ public Map getDynamoUpdateLeaseUpdate(Lease lease, UpdateField updateField) {
+ // There is no application-specific data in Lease - just return a map that increments the counter.
+ return new HashMap();
+ }
+
@Override
public Collection getKeySchema() {
List keySchema = new ArrayList();
diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/UpdateField.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/UpdateField.java
new file mode 100644
index 00000000..d621999b
--- /dev/null
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/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 com.amazonaws.services.kinesis.leases.impl;
+
+/**
+ * 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/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java
index 4de54607..1b63bd01 100644
--- a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java
@@ -20,6 +20,7 @@ import com.amazonaws.services.kinesis.leases.exceptions.DependencyException;
import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException;
import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException;
import com.amazonaws.services.kinesis.leases.impl.Lease;
+import com.amazonaws.services.kinesis.leases.impl.UpdateField;
/**
* Supports basic CRUD operations for Leases.
@@ -180,6 +181,19 @@ public interface ILeaseManager {
public boolean updateLease(T 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.
+ **
+ * @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(T lease, UpdateField updateField)
+ throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+ throw new UnsupportedOperationException("updateLeaseWithMetaInfo is not implemented.");
+ }
+
/**
* Check (synchronously) if there are any leases in the lease table.
*
diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java
index 35a8fc15..2d9ea0c9 100644
--- a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java
+++ b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java
@@ -23,6 +23,7 @@ import com.amazonaws.services.dynamodbv2.model.AttributeValueUpdate;
import com.amazonaws.services.dynamodbv2.model.ExpectedAttributeValue;
import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
import com.amazonaws.services.kinesis.leases.impl.Lease;
+import com.amazonaws.services.kinesis.leases.impl.UpdateField;
/**
* Utility class that manages the mapping of Lease objects/operations to records in DynamoDB.
@@ -78,6 +79,13 @@ public interface ILeaseSerializer {
*/
public Map getDynamoNonexistantExpectation();
+ /**
+ * @return the attribute value map asserting that a lease does exist.
+ */
+ default Map getDynamoExistentExpectation(final String leaseKey) {
+ throw new UnsupportedOperationException("DynamoExistentExpectation is not implemented");
+ }
+
/**
* @param lease
* @return the attribute value map that increments a lease counter
@@ -104,6 +112,15 @@ public interface ILeaseSerializer {
*/
public Map getDynamoUpdateLeaseUpdate(T lease);
+ /**
+ * @param lease
+ * @param updateField
+ * @return the attribute value map that updates application-specific data for a lease
+ */
+ default Map getDynamoUpdateLeaseUpdate(T lease, UpdateField updateField) {
+ throw new UnsupportedOperationException();
+ }
+
/**
* @return the key schema for creating a DynamoDB table to store leases
*/
diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java
index d9160f0f..6a5e76b9 100644
--- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java
+++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java
@@ -124,7 +124,7 @@ public class ConsumerStatesTest {
assertThat(state.successTransition(), equalTo(ShardConsumerState.INITIALIZING.getConsumerState()));
for (ShutdownReason shutdownReason : ShutdownReason.values()) {
assertThat(state.shutdownTransition(shutdownReason),
- equalTo(ShardConsumerState.SHUTDOWN_COMPLETE.getConsumerState()));
+ equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState()));
}
assertThat(state.getState(), equalTo(ShardConsumerState.WAITING_ON_PARENT_SHARDS));
diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java
index c8e820bb..7f53133b 100644
--- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java
+++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java
@@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import java.util.Arrays;
import java.util.List;
+import com.amazonaws.services.kinesis.leases.impl.UpdateField;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -53,6 +54,7 @@ class ExceptionThrowingLeaseManager implements ILeaseManager
DELETELEASE(9),
DELETEALL(10),
UPDATELEASE(11),
+ UPDATELEASEWITHMETAINFO(12),
NONE(Integer.MIN_VALUE);
private Integer index;
@@ -197,6 +199,14 @@ class ExceptionThrowingLeaseManager implements ILeaseManager
return leaseManager.updateLease(lease);
}
+ @Override
+ public void updateLeaseWithMetaInfo(KinesisClientLease lease, UpdateField updateField)
+ throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+ throwExceptions("updateLeaseWithMetaInfo", ExceptionThrowingLeaseManagerMethods.UPDATELEASEWITHMETAINFO);
+
+ leaseManager.updateLeaseWithMetaInfo(lease, updateField);
+ }
+
@Override
public KinesisClientLease getLease(String shardId)
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
@@ -215,7 +225,7 @@ class ExceptionThrowingLeaseManager implements ILeaseManager
@Override
public boolean isLeaseTableEmpty() throws DependencyException,
InvalidStateException, ProvisionedThroughputException {
- return false;
+ return leaseManager.listLeases().isEmpty();
}
}
diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java
index 7a125f99..a79fc9b2 100644
--- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java
+++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java
@@ -36,6 +36,7 @@ import java.util.Collections;
import java.util.Date;
import java.util.List;
+import com.amazonaws.services.kinesis.model.ChildShard;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -132,7 +133,7 @@ public class KinesisDataFetcherTest {
}
@Test
- public void testadvanceIteratorTo() throws KinesisClientLibException {
+ public void testadvanceIteratorTo() throws Exception {
IKinesisProxy kinesis = mock(IKinesisProxy.class);
ICheckpoint checkpoint = mock(ICheckpoint.class);
@@ -146,9 +147,13 @@ public class KinesisDataFetcherTest {
GetRecordsResult outputA = new GetRecordsResult();
List recordsA = new ArrayList();
outputA.setRecords(recordsA);
+ outputA.setNextShardIterator("nextShardIteratorA");
+ outputA.setChildShards(Collections.emptyList());
GetRecordsResult outputB = new GetRecordsResult();
List recordsB = new ArrayList();
outputB.setRecords(recordsB);
+ outputB.setNextShardIterator("nextShardIteratorB");
+ outputB.setChildShards(Collections.emptyList());
when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqA)).thenReturn(iteratorA);
when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqB)).thenReturn(iteratorB);
@@ -166,7 +171,7 @@ public class KinesisDataFetcherTest {
}
@Test
- public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() {
+ public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() throws Exception{
IKinesisProxy kinesis = mock(IKinesisProxy.class);
KinesisDataFetcher fetcher = new KinesisDataFetcher(kinesis, SHARD_INFO);
@@ -189,7 +194,7 @@ public class KinesisDataFetcherTest {
}
@Test
- public void testGetRecordsWithResourceNotFoundException() {
+ public void testGetRecordsWithResourceNotFoundException() throws Exception {
// Set up arguments used by proxy
String nextIterator = "TestShardIterator";
int maxRecords = 100;
@@ -211,11 +216,12 @@ public class KinesisDataFetcherTest {
}
@Test
- public void testNonNullGetRecords() {
+ public void testNonNullGetRecords() throws Exception {
String nextIterator = "TestIterator";
int maxRecords = 100;
KinesisProxy mockProxy = mock(KinesisProxy.class);
+ when(mockProxy.getIterator(anyString(), anyString())).thenReturn("targetIterator");
doThrow(new ResourceNotFoundException("Test Exception")).when(mockProxy).get(nextIterator, maxRecords);
KinesisDataFetcher dataFetcher = new KinesisDataFetcher(mockProxy, SHARD_INFO);
@@ -232,17 +238,25 @@ public class KinesisDataFetcherTest {
final String NEXT_ITERATOR_ONE = "NextIteratorOne";
final String NEXT_ITERATOR_TWO = "NextIteratorTwo";
when(kinesisProxy.getIterator(anyString(), anyString())).thenReturn(INITIAL_ITERATOR);
- GetRecordsResult iteratorOneResults = mock(GetRecordsResult.class);
- when(iteratorOneResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_ONE);
+
+ GetRecordsResult iteratorOneResults = new GetRecordsResult();
+ iteratorOneResults.setNextShardIterator(NEXT_ITERATOR_ONE);
+ iteratorOneResults.setChildShards(Collections.emptyList());
when(kinesisProxy.get(eq(INITIAL_ITERATOR), anyInt())).thenReturn(iteratorOneResults);
- GetRecordsResult iteratorTwoResults = mock(GetRecordsResult.class);
+ GetRecordsResult iteratorTwoResults = new GetRecordsResult();
+ iteratorTwoResults.setNextShardIterator(NEXT_ITERATOR_TWO);
+ iteratorTwoResults.setChildShards(Collections.emptyList());
when(kinesisProxy.get(eq(NEXT_ITERATOR_ONE), anyInt())).thenReturn(iteratorTwoResults);
- when(iteratorTwoResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_TWO);
- GetRecordsResult finalResult = mock(GetRecordsResult.class);
+ GetRecordsResult finalResult = new GetRecordsResult();
+ finalResult.setNextShardIterator(null);
+ List childShards = new ArrayList<>();
+ ChildShard childShard = new ChildShard();
+ childShard.setParentShards(Collections.singletonList("parentShardId"));
+ childShards.add(childShard);
+ finalResult.setChildShards(childShards);
when(kinesisProxy.get(eq(NEXT_ITERATOR_TWO), anyInt())).thenReturn(finalResult);
- when(finalResult.getNextShardIterator()).thenReturn(null);
KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO);
dataFetcher.initialize("TRIM_HORIZON",
@@ -276,13 +290,14 @@ public class KinesisDataFetcherTest {
}
@Test
- public void testRestartIterator() {
+ public void testRestartIterator() throws Exception{
GetRecordsResult getRecordsResult = mock(GetRecordsResult.class);
- GetRecordsResult restartGetRecordsResult = new GetRecordsResult();
+ GetRecordsResult restartGetRecordsResult = mock(GetRecordsResult.class);
Record record = mock(Record.class);
final String initialIterator = "InitialIterator";
final String nextShardIterator = "NextShardIterator";
final String restartShardIterator = "RestartIterator";
+ final String restartNextShardIterator = "RestartNextIterator";
final String sequenceNumber = "SequenceNumber";
final String iteratorType = "AT_SEQUENCE_NUMBER";
KinesisProxy kinesisProxy = mock(KinesisProxy.class);
@@ -292,6 +307,7 @@ public class KinesisDataFetcherTest {
when(kinesisProxy.get(eq(initialIterator), eq(10))).thenReturn(getRecordsResult);
when(getRecordsResult.getRecords()).thenReturn(Collections.singletonList(record));
when(getRecordsResult.getNextShardIterator()).thenReturn(nextShardIterator);
+ when(getRecordsResult.getChildShards()).thenReturn(Collections.emptyList());
when(record.getSequenceNumber()).thenReturn(sequenceNumber);
fetcher.initialize(InitialPositionInStream.LATEST.toString(), INITIAL_POSITION_LATEST);
@@ -300,6 +316,8 @@ public class KinesisDataFetcherTest {
verify(kinesisProxy).get(eq(initialIterator), eq(10));
when(kinesisProxy.getIterator(eq(SHARD_ID), eq(iteratorType), eq(sequenceNumber))).thenReturn(restartShardIterator);
+ when(restartGetRecordsResult.getNextShardIterator()).thenReturn(restartNextShardIterator);
+ when(restartGetRecordsResult.getChildShards()).thenReturn(Collections.emptyList());
when(kinesisProxy.get(eq(restartShardIterator), eq(10))).thenReturn(restartGetRecordsResult);
fetcher.restartIterator();
@@ -309,7 +327,7 @@ public class KinesisDataFetcherTest {
}
@Test (expected = IllegalStateException.class)
- public void testRestartIteratorNotInitialized() {
+ public void testRestartIteratorNotInitialized() throws Exception {
KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO);
dataFetcher.restartIterator();
}
@@ -354,6 +372,8 @@ public class KinesisDataFetcherTest {
List expectedRecords = new ArrayList();
GetRecordsResult response = new GetRecordsResult();
response.setRecords(expectedRecords);
+ response.setNextShardIterator("testNextShardIterator");
+ response.setChildShards(Collections.emptyList());
when(kinesis.getIterator(SHARD_ID, initialPositionInStream.getTimestamp())).thenReturn(iterator);
when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqNo)).thenReturn(iterator);
diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java
new file mode 100644
index 00000000..779ba92f
--- /dev/null
+++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java
@@ -0,0 +1,616 @@
+/*
+ * 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 com.amazonaws.services.kinesis.clientlibrary.lib.worker;
+
+import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy;
+import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
+import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease;
+import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
+import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
+import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
+import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory;
+import com.amazonaws.services.kinesis.model.HashKeyRange;
+import com.amazonaws.services.kinesis.model.Shard;
+import com.amazonaws.util.CollectionUtils;
+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 java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MAX_HASH_KEY;
+import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MIN_HASH_KEY;
+import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.deserialize;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class PeriodicShardSyncManagerTest {
+
+ private static final String WORKER_ID = "workerId";
+ public static final long LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS = 2 * 60 * 1000L;
+ public static final int LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD = 3;
+
+ /** Manager for PERIODIC shard sync strategy */
+ private PeriodicShardSyncManager periodicShardSyncManager;
+
+ /** Manager for SHARD_END shard sync strategy */
+ private PeriodicShardSyncManager auditorPeriodicShardSyncManager;
+
+ @Mock
+ private LeaderDecider leaderDecider;
+ @Mock
+ private ShardSyncTask shardSyncTask;
+ @Mock
+ private ILeaseManager leaseManager;
+ @Mock
+ private IKinesisProxy kinesisProxy;
+
+ private IMetricsFactory metricsFactory = new NullMetricsFactory();
+
+ @Before
+ public void setup() {
+ periodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask,
+ metricsFactory, leaseManager, kinesisProxy, false, LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS,
+ LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD);
+ auditorPeriodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask,
+ metricsFactory, leaseManager, kinesisProxy, true, LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS,
+ LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD);
+ }
+
+ @Test
+ public void testForFailureWhenHashRangesAreIncomplete() {
+ List hashRanges = new ArrayList() {{
+ add(deserialize(MIN_HASH_KEY.toString(), "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 -> {
+ KinesisClientLease lease = new KinesisClientLease();
+ lease.setHashKeyRange(hashKeyRangeForLease);
+ lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
+ return lease;
+ }).collect(Collectors.toList());
+ Assert.assertTrue(PeriodicShardSyncManager
+ .checkForHoleInHashKeyRanges(hashRanges).isPresent());
+ }
+
+ @Test
+ public void testForSuccessWhenHashRangesAreComplete() {
+ List hashRanges = 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 -> {
+ KinesisClientLease lease = new KinesisClientLease();
+ lease.setHashKeyRange(hashKeyRangeForLease);
+ lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
+ return lease;
+ }).collect(Collectors.toList());
+ Assert.assertFalse(PeriodicShardSyncManager
+ .checkForHoleInHashKeyRanges(hashRanges).isPresent());
+ }
+
+ @Test
+ public void testForSuccessWhenUnsortedHashRangesAreComplete() {
+ List hashRanges = new ArrayList() {{
+ add(deserialize("4", "23"));
+ add(deserialize("2", "3"));
+ add(deserialize(MIN_HASH_KEY.toString(), "1"));
+ add(deserialize("24", MAX_HASH_KEY.toString()));
+ add(deserialize("6", "23"));
+ }}.stream().map(hashKeyRangeForLease -> {
+ KinesisClientLease lease = new KinesisClientLease();
+ lease.setHashKeyRange(hashKeyRangeForLease);
+ lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
+ return lease;
+ }).collect(Collectors.toList());
+ Assert.assertFalse(PeriodicShardSyncManager
+ .checkForHoleInHashKeyRanges(hashRanges).isPresent());
+ }
+
+ @Test
+ public void testForSuccessWhenHashRangesAreCompleteForOverlappingLeasesAtEnd() {
+ List hashRanges = new ArrayList