From 0a2883770498565aab68199db3995823370b0ebf Mon Sep 17 00:00:00 2001 From: Mathieu Fortin Date: Tue, 18 Oct 2022 10:12:21 -0400 Subject: [PATCH 01/88] Evict lease on shutdown (#1) --- .gitignore | 3 +- .../kinesis/common/CommonCalculations.java | 18 ++ .../amazon/kinesis/coordinator/Scheduler.java | 3 +- .../software/amazon/kinesis/leases/Lease.java | 27 ++- .../kinesis/leases/LeaseManagementConfig.java | 14 ++ .../dynamodb/DynamoDBLeaseCoordinator.java | 9 +- .../DynamoDBLeaseManagementFactory.java | 29 +-- .../leases/dynamodb/DynamoDBLeaseTaker.java | 49 +++-- .../kinesis/lifecycle/ShardConsumer.java | 28 ++- .../lifecycle/ShardConsumerArgument.java | 1 + .../kinesis/common/CommonCalculationTest.java | 46 ++++ .../leases/LeaseCoordinatorExerciser.java | 3 +- ...namoDBLeaseCoordinatorIntegrationTest.java | 41 +--- .../DynamoDBLeaseCoordinatorTest.java | 5 +- .../DynamoDBLeaseTakerIntegrationTest.java | 17 +- .../leases/dynamodb/TestHarnessBuilder.java | 6 + .../kinesis/lifecycle/ConsumerStatesTest.java | 3 +- .../kinesis/lifecycle/ShardConsumerTest.java | 198 +++++++++++++++++- 18 files changed, 402 insertions(+), 98 deletions(-) create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/CommonCalculationTest.java diff --git a/.gitignore b/.gitignore index 973f5095..5454f215 100644 --- a/.gitignore +++ b/.gitignore @@ -2,4 +2,5 @@ target/ AwsCredentials.properties .idea *.iml - +.sdkmanrc +.vscode diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/CommonCalculations.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/CommonCalculations.java index edb6de2e..94328db2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/CommonCalculations.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/CommonCalculations.java @@ -28,4 +28,22 @@ public class CommonCalculations { public static long getRenewerTakerIntervalMillis(long leaseDurationMillis, long epsilonMillis) { return leaseDurationMillis / 3 - epsilonMillis; } + + /** + * Convenience method for calculating lease taker intervals in milliseconds. + * + * @param leaseTakerIntervalMillis Current value for interval (from default or overriden). + * @param leaseDurationMillis Duration of a lease + * @param epsilonMillis Allow for some variance when calculating lease expirations + * @return lease taker interval. + */ + public static long getLeaseTakerIntervalMillis( + long leaseTakerIntervalMillis, long leaseDurationMillis, long epsilonMillis + ) { + if (leaseTakerIntervalMillis > 0) { + return leaseTakerIntervalMillis; + } + + return (leaseDurationMillis + epsilonMillis) * 2; + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 5d9f73e9..6f1486ad 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -945,7 +945,8 @@ public class Scheduler implements Runnable { hierarchicalShardSyncerProvider.apply(streamConfig), metricsFactory, leaseCleanupManager, - schemaRegistryDecoder + schemaRegistryDecoder, + leaseManagementConfig.evictLeaseOnShutdown() ); return new ShardConsumer(cache, executorService, shardInfo, lifecycleConfig.logWarningForTaskAfterMillis(), argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java index 4074db22..8ea522ab 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java @@ -44,7 +44,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; public class Lease { /* * See javadoc for System.nanoTime - summary: - * + * * Sometimes System.nanoTime's return values will wrap due to overflow. When they do, the difference between two * values will be very large. We will consider leases to be expired if they are more than a year old. */ @@ -111,7 +111,7 @@ public class Lease { /** * Copy constructor, used by clone(). - * + * * @param lease lease to copy */ protected Lease(Lease lease) { @@ -164,7 +164,7 @@ public class Lease { /** * Updates this Lease's mutable, application-specific fields based on the passed-in lease object. Does not update * fields that are internal to the leasing library (leaseKey, leaseOwner, leaseCounter). - * + * * @param lease */ public void update(final Lease lease) { @@ -195,9 +195,16 @@ public class Lease { } } + /** + * @return true if this lease is unassigned (no assigned owner), false otherwise. + */ + public boolean isUnassigned() { + return leaseOwner == null; + } + /** * Sets lastCounterIncrementNanos - * + * * @param lastCounterIncrementNanos last renewal in nanoseconds since the epoch */ public void lastCounterIncrementNanos(Long lastCounterIncrementNanos) { @@ -206,7 +213,7 @@ public class Lease { /** * Sets concurrencyToken. - * + * * @param concurrencyToken may not be null */ public void concurrencyToken(@NonNull final UUID concurrencyToken) { @@ -215,7 +222,7 @@ public class Lease { /** * Sets leaseKey. LeaseKey is immutable once set. - * + * * @param leaseKey may not be null. */ public void leaseKey(@NonNull final String leaseKey) { @@ -227,7 +234,7 @@ public class Lease { /** * Sets leaseCounter. - * + * * @param leaseCounter may not be null */ public void leaseCounter(@NonNull final Long leaseCounter) { @@ -303,7 +310,7 @@ public class Lease { /** * Sets leaseOwner. - * + * * @param leaseOwner may be null. */ public void leaseOwner(String leaseOwner) { @@ -312,12 +319,10 @@ public class Lease { /** * Returns a deep copy of this object. Type-unsafe - there aren't good mechanisms for copy-constructing generics. - * + * * @return A deep copy of this object. */ public Lease copy() { return new Lease(this); } - - } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java index 89e6a3bf..e46ce8c7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java @@ -224,6 +224,18 @@ public class LeaseManagementConfig { private long listShardsCacheAllowedAgeInSeconds = 30; private int cacheMissWarningModulus = 250; + /** + * Interval at which the lease taker will execute. + * If unspecified, an interval will be calculated based on the lease duration. + */ + private long leaseTakerIntervalMillis = -1L; + + /** + * If leases should be evicted or not on shutdown requested. + * By default, leases are not evicted. + */ + private boolean evictLeaseOnShutdown = false; + private MetricsFactory metricsFactory = new NullMetricsFactory(); @Deprecated @@ -326,6 +338,7 @@ public class LeaseManagementConfig { initialPositionInStream(), failoverTimeMillis(), epsilonMillis(), + leaseTakerIntervalMillis, maxLeasesForWorker(), maxLeasesToStealAtOneTime(), maxLeaseRenewalThreads(), @@ -361,6 +374,7 @@ public class LeaseManagementConfig { executorService(), failoverTimeMillis(), epsilonMillis(), + leaseTakerIntervalMillis, maxLeasesForWorker(), maxLeasesToStealAtOneTime(), maxLeaseRenewalThreads(), diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java index 07e9068d..ee9c2ce2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java @@ -49,6 +49,7 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import static software.amazon.kinesis.common.CommonCalculations.getRenewerTakerIntervalMillis; +import static software.amazon.kinesis.common.CommonCalculations.getLeaseTakerIntervalMillis; /** * LeaseCoordinator abstracts away LeaseTaker and LeaseRenewer from the application code that's using leasing. It owns @@ -108,12 +109,13 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { final String workerIdentifier, final long leaseDurationMillis, final long epsilonMillis, + final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewerThreadCount, final MetricsFactory metricsFactory) { - this(leaseRefresher, workerIdentifier, leaseDurationMillis, epsilonMillis, maxLeasesForWorker, - maxLeasesToStealAtOneTime, maxLeaseRenewerThreadCount, + this(leaseRefresher, workerIdentifier, leaseDurationMillis, epsilonMillis, leaseTakerIntervalMillis, + maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewerThreadCount, TableConstants.DEFAULT_INITIAL_LEASE_TABLE_READ_CAPACITY, TableConstants.DEFAULT_INITIAL_LEASE_TABLE_WRITE_CAPACITY, metricsFactory); } @@ -144,6 +146,7 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { final String workerIdentifier, final long leaseDurationMillis, final long epsilonMillis, + final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewerThreadCount, @@ -158,7 +161,7 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { this.leaseRenewer = new DynamoDBLeaseRenewer( leaseRefresher, workerIdentifier, leaseDurationMillis, leaseRenewalThreadpool, metricsFactory); this.renewerIntervalMillis = getRenewerTakerIntervalMillis(leaseDurationMillis, epsilonMillis); - this.takerIntervalMillis = (leaseDurationMillis + epsilonMillis) * 2; + this.takerIntervalMillis = getLeaseTakerIntervalMillis(leaseTakerIntervalMillis, leaseDurationMillis, epsilonMillis); if (initialLeaseTableReadCapacity <= 0) { throw new IllegalArgumentException("readCapacity should be >= 1"); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java index ad1a2300..cffdcaa8 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -68,6 +68,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { private final long failoverTimeMillis; private final long epsilonMillis; + private final long leaseTakerIntervalMillis; private final int maxLeasesForWorker; private final int maxLeasesToStealAtOneTime; private final int maxLeaseRenewalThreads; @@ -119,14 +120,14 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, - final long failoverTimeMillis, final long epsilonMillis, final int maxLeasesForWorker, + final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, - initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + initialPositionInStream, failoverTimeMillis, epsilonMillis, leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, @@ -169,7 +170,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, - final long failoverTimeMillis, final long epsilonMillis, final int maxLeasesForWorker, + final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, @@ -177,7 +178,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, - initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + initialPositionInStream, failoverTimeMillis, epsilonMillis, leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, @@ -219,7 +220,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, - final long failoverTimeMillis, final long epsilonMillis, final int maxLeasesForWorker, + final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, @@ -228,7 +229,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, - initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + initialPositionInStream, failoverTimeMillis, epsilonMillis, leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, @@ -270,7 +271,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, - final long failoverTimeMillis, final long epsilonMillis, final int maxLeasesForWorker, + final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, @@ -280,7 +281,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, - initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + initialPositionInStream, failoverTimeMillis, epsilonMillis, leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, @@ -323,7 +324,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, - final long failoverTimeMillis, final long epsilonMillis, final int maxLeasesForWorker, + final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, @@ -334,7 +335,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { Duration dynamoDbRequestTimeout, BillingMode billingMode) { this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName, - workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, @@ -374,7 +375,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { */ private DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final StreamConfig streamConfig, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, - final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, + final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, @@ -384,7 +385,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { this(kinesisClient, dynamoDBClient, tableName, - workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, @@ -428,7 +429,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { */ public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, - final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, + final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, final long leaseTakerIntervalMillis, final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, @@ -446,6 +447,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.executorService = executorService; this.failoverTimeMillis = failoverTimeMillis; this.epsilonMillis = epsilonMillis; + this.leaseTakerIntervalMillis = leaseTakerIntervalMillis; this.maxLeasesForWorker = maxLeasesForWorker; this.maxLeasesToStealAtOneTime = maxLeasesToStealAtOneTime; this.maxLeaseRenewalThreads = maxLeaseRenewalThreads; @@ -476,6 +478,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { workerIdentifier, failoverTimeMillis, epsilonMillis, + leaseTakerIntervalMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java index a90ef56e..5f7eb6b1 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java @@ -191,9 +191,9 @@ public class DynamoDBLeaseTaker implements LeaseTaker { return takenLeases; } - List expiredLeases = getExpiredLeases(); + List availableLeases = getAvailableLeases(); - Set leasesToTake = computeLeasesToTake(expiredLeases); + Set leasesToTake = computeLeasesToTake(availableLeases); leasesToTake = updateStaleLeasesWithLatestState(updateAllLeasesTotalTimeMillis, leasesToTake); Set untakenLeaseKeys = new HashSet<>(); @@ -358,34 +358,43 @@ public class DynamoDBLeaseTaker implements LeaseTaker { } /** - * @return list of leases that were expired as of our last scan. + * @return list of leases that are available for the taking. + * Expired leases and orphan leases (without owner) are considered available. */ - private List getExpiredLeases() { - List expiredLeases = new ArrayList<>(); + private List getAvailableLeases() { + List availableLeases = new ArrayList<>(); for (Lease lease : allLeases.values()) { - if (lease.isExpired(leaseDurationNanos, lastScanTimeNanos)) { - expiredLeases.add(lease); + if (isExpired(lease) || isUnassigned(lease)) { + availableLeases.add(lease); } } - return expiredLeases; + return availableLeases; + } + + private boolean isUnassigned(Lease lease) { + return lease.isUnassigned(); + } + + private boolean isExpired(Lease lease) { + return lease.isExpired(leaseDurationNanos, lastScanTimeNanos); } /** * Compute the number of leases I should try to take based on the state of the system. * - * @param expiredLeases list of leases we determined to be expired + * @param availableLeases list of leases we determined to be available * @return set of leases to take. */ - private Set computeLeasesToTake(List expiredLeases) { - Map leaseCounts = computeLeaseCounts(expiredLeases); + private Set computeLeasesToTake(List availableLeases) { + Map leaseCounts = computeLeaseCounts(availableLeases); Set leasesToTake = new HashSet<>(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION); MetricsUtil.addWorkerIdentifier(scope, workerIdentifier); List veryOldLeases = new ArrayList<>(); - final int numAvailableLeases = expiredLeases.size(); + final int numAvailableLeases = availableLeases.size(); int numLeases = 0; int numWorkers = 0; int numLeasesToReachTarget = 0; @@ -454,16 +463,16 @@ public class DynamoDBLeaseTaker implements LeaseTaker { return leasesToTake; } - // Shuffle expiredLeases so workers don't all try to contend for the same leases. - Collections.shuffle(expiredLeases); + // Shuffle availableLeases so workers don't all try to contend for the same leases. + Collections.shuffle(availableLeases); - if (expiredLeases.size() > 0) { - // If we have expired leases, get up to leases from expiredLeases - for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) { - leasesToTake.add(expiredLeases.remove(0)); + if (availableLeases.size() > 0) { + // If we have available leases, get up to leases from availableLeases + for (; numLeasesToReachTarget > 0 && availableLeases.size() > 0; numLeasesToReachTarget--) { + leasesToTake.add(availableLeases.remove(0)); } } else { - // If there are no expired leases and we need a lease, consider stealing. + // If there are no available leases and we need a lease, consider stealing. List leasesToSteal = chooseLeasesToSteal(leaseCounts, numLeasesToReachTarget, target); for (Lease leaseToSteal : leasesToSteal) { log.info("Worker {} needed {} leases but none were expired, so it will steal lease {} from {}", @@ -482,7 +491,7 @@ public class DynamoDBLeaseTaker implements LeaseTaker { } } finally { - scope.addData("ExpiredLeases", expiredLeases.size(), StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("AvailableLeases", availableLeases.size(), StandardUnit.COUNT, MetricsLevel.SUMMARY); scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY); scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED); scope.addData("NeededLeases", Math.max(numLeasesToReachTarget, 0), StandardUnit.COUNT, MetricsLevel.DETAILED); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java index b6e7c068..442c35e6 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java @@ -16,7 +16,6 @@ package software.amazon.kinesis.lifecycle; import java.time.Duration; import java.time.Instant; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -24,8 +23,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; import java.util.function.Function; -import org.reactivestreams.Subscription; - import com.google.common.annotations.VisibleForTesting; import lombok.AccessLevel; @@ -33,10 +30,18 @@ import lombok.Getter; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; -import software.amazon.awssdk.services.kinesis.model.ChildShard; + +import org.reactivestreams.Subscription; + import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCoordinator; +import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardInfo; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.lifecycle.events.TaskExecutionListenerInput; import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; @@ -436,6 +441,21 @@ public class ShardConsumer { if (shutdownReason == null || shutdownReason.canTransitionTo(reason)) { shutdownReason = reason; } + + if (reason == ShutdownReason.REQUESTED && shardConsumerArgument.evictLeaseOnShutdown()) { + log.debug("{} : Shutdown({}): Evicting lease.", streamIdentifier, shardInfo.shardId()); + + LeaseCoordinator leaseCoordinator = shardConsumerArgument.leaseCoordinator(); + LeaseRefresher leaseRefresher = leaseCoordinator.leaseRefresher(); + Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(ShardInfo.getLeaseKey(shardInfo)); + + try { + leaseRefresher.evictLease(currentShardLease); + } catch (DependencyException | InvalidStateException | ProvisionedThroughputException ex) { + // This is recoverable. This lease will expire and some other consumer will take it. + log.error("Could not evict lease on requested shutdown of {}", shardInfo.shardId(), ex); + } + } } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java index 0518b830..43478d9c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java @@ -75,4 +75,5 @@ public class ShardConsumerArgument { private final MetricsFactory metricsFactory; private final LeaseCleanupManager leaseCleanupManager; private final SchemaRegistryDecoder schemaRegistryDecoder; + private final boolean evictLeaseOnShutdown; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/CommonCalculationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/CommonCalculationTest.java new file mode 100644 index 00000000..4ae87834 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/CommonCalculationTest.java @@ -0,0 +1,46 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.common; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class CommonCalculationTest { + + @Test + public void testGetLeaseTakerIntervalMillisWithDefault() { + long leaseTakerIntervalMillis = -1L; + long leaseDurationMillis = 10000L; + long epsilonMillis = 25L; + + long expected = (leaseDurationMillis + epsilonMillis) * 2; + + assertEquals(expected, CommonCalculations.getLeaseTakerIntervalMillis( + leaseTakerIntervalMillis, leaseDurationMillis, epsilonMillis)); + } + + @Test + public void testGetLeaseTakerIntervalMillisWhenOverriden() { + long leaseTakerIntervalMillis = 200L; + long leaseDurationMillis = 10000L; + long epsilonMillis = 25L; + + long expected = leaseTakerIntervalMillis; + + assertEquals(expected, CommonCalculations.getLeaseTakerIntervalMillis( + leaseTakerIntervalMillis, leaseDurationMillis, epsilonMillis)); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java index 186fe290..9ba49d0d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java @@ -46,6 +46,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @Slf4j public class LeaseCoordinatorExerciser { + private static final long LEASE_TAKER_INTERVAL_MILLIS = -1L; private static final int MAX_LEASES_FOR_WORKER = Integer.MAX_VALUE; private static final int MAX_LEASES_TO_STEAL_AT_ONE_TIME = 1; private static final int MAX_LEASE_RENEWER_THREAD_COUNT = 20; @@ -85,7 +86,7 @@ public class LeaseCoordinatorExerciser { String workerIdentifier = "worker-" + Integer.toString(i); LeaseCoordinator coord = new DynamoDBLeaseCoordinator(leaseRefresher, workerIdentifier, leaseDurationMillis, - epsilonMillis, MAX_LEASES_FOR_WORKER, MAX_LEASES_TO_STEAL_AT_ONE_TIME, + epsilonMillis, LEASE_TAKER_INTERVAL_MILLIS, MAX_LEASES_FOR_WORKER, MAX_LEASES_TO_STEAL_AT_ONE_TIME, MAX_LEASE_RENEWER_THREAD_COUNT, INITIAL_LEASE_TABLE_READ_CAPACITY, INITIAL_LEASE_TABLE_WRITE_CAPACITY, metricsFactory); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java index d89c010e..2a3b6827 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java @@ -20,9 +20,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -34,11 +32,10 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.runners.MockitoJUnitRunner; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; -import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.kinesis.checkpoint.dynamodb.DynamoDBCheckpointer; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; +import software.amazon.kinesis.leases.LeaseIntegrationTest; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.LeasingException; @@ -48,21 +45,19 @@ import software.amazon.kinesis.metrics.NullMetricsFactory; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @RunWith(MockitoJUnitRunner.class) -public class DynamoDBLeaseCoordinatorIntegrationTest { - private static final int ATTEMPTS = 20; +public class DynamoDBLeaseCoordinatorIntegrationTest extends LeaseIntegrationTest { private static final String OPERATION = "TestOperation"; - private static final String TABLE_NAME = DynamoDBLeaseCoordinatorIntegrationTest.class.getSimpleName(); private static final String WORKER_ID = UUID.randomUUID().toString(); private static final long LEASE_DURATION_MILLIS = 5000L; private static final long EPSILON_MILLIS = 25L; + private static final long LEASE_TAKER_INTERVAL_MILLIS = -1L; private static final int MAX_LEASES_FOR_WORKER = Integer.MAX_VALUE; private static final int MAX_LEASES_TO_STEAL_AT_ONE_TIME = 1; private static final int MAX_LEASE_RENEWER_THREAD_COUNT = 20; private static final long INITIAL_LEASE_TABLE_READ_CAPACITY = 10L; private static final long INITIAL_LEASE_TABLE_WRITE_CAPACITY = 10L; - private static DynamoDBLeaseRefresher leaseRefresher; private static DynamoDBCheckpointer dynamoDBCheckpointer; private LeaseCoordinator coordinator; @@ -71,34 +66,8 @@ public class DynamoDBLeaseCoordinatorIntegrationTest { @Before public void setup() throws ProvisionedThroughputException, DependencyException, InvalidStateException { - final boolean useConsistentReads = true; - if (leaseRefresher == null) { - DynamoDbAsyncClient dynamoDBClient = DynamoDbAsyncClient.builder() - .credentialsProvider(DefaultCredentialsProvider.create()).build(); - leaseRefresher = new DynamoDBLeaseRefresher(TABLE_NAME, dynamoDBClient, new DynamoDBLeaseSerializer(), - useConsistentReads, TableCreatorCallback.NOOP_TABLE_CREATOR_CALLBACK); - } - leaseRefresher.createLeaseTableIfNotExists(10L, 10L); - - int retryLeft = ATTEMPTS; - - while (!leaseRefresher.leaseTableExists()) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - // Sleep called. - } - retryLeft--; - if (retryLeft == 0) { - if (!leaseRefresher.leaseTableExists()) { - fail("Failed to create table"); - } - } - } - - leaseRefresher.deleteAll(); - coordinator = new DynamoDBLeaseCoordinator(leaseRefresher, WORKER_ID, LEASE_DURATION_MILLIS, - EPSILON_MILLIS, MAX_LEASES_FOR_WORKER, MAX_LEASES_TO_STEAL_AT_ONE_TIME, MAX_LEASE_RENEWER_THREAD_COUNT, + coordinator = new DynamoDBLeaseCoordinator(leaseRefresher, WORKER_ID, LEASE_DURATION_MILLIS, EPSILON_MILLIS, + LEASE_TAKER_INTERVAL_MILLIS, MAX_LEASES_FOR_WORKER, MAX_LEASES_TO_STEAL_AT_ONE_TIME, MAX_LEASE_RENEWER_THREAD_COUNT, INITIAL_LEASE_TABLE_READ_CAPACITY, INITIAL_LEASE_TABLE_WRITE_CAPACITY, metricsFactory); dynamoDBCheckpointer = new DynamoDBCheckpointer(coordinator, leaseRefresher); dynamoDBCheckpointer.operation(OPERATION); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java index caa7a6c7..93ac718b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java @@ -22,6 +22,7 @@ public class DynamoDBLeaseCoordinatorTest { private static final String WORKER_ID = UUID.randomUUID().toString(); private static final long LEASE_DURATION_MILLIS = 5000L; private static final long EPSILON_MILLIS = 25L; + private static final long LEASE_TAKER_INTERVAL_MILLIS = -1L; private static final int MAX_LEASES_FOR_WORKER = Integer.MAX_VALUE; private static final int MAX_LEASES_TO_STEAL_AT_ONE_TIME = 1; private static final int MAX_LEASE_RENEWER_THREAD_COUNT = 20; @@ -39,8 +40,8 @@ public class DynamoDBLeaseCoordinatorTest { @Before public void setup() { - this.leaseCoordinator = new DynamoDBLeaseCoordinator(leaseRefresher, WORKER_ID, LEASE_DURATION_MILLIS, - EPSILON_MILLIS, MAX_LEASES_FOR_WORKER, MAX_LEASES_TO_STEAL_AT_ONE_TIME, MAX_LEASE_RENEWER_THREAD_COUNT, + this.leaseCoordinator = new DynamoDBLeaseCoordinator(leaseRefresher, WORKER_ID, LEASE_DURATION_MILLIS, EPSILON_MILLIS, + LEASE_TAKER_INTERVAL_MILLIS, MAX_LEASES_FOR_WORKER, MAX_LEASES_TO_STEAL_AT_ONE_TIME, MAX_LEASE_RENEWER_THREAD_COUNT, INITIAL_LEASE_TABLE_READ_CAPACITY, INITIAL_LEASE_TABLE_WRITE_CAPACITY, metricsFactory); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java index 475f1940..02875852 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java @@ -88,6 +88,21 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest { builder.takeMutateAssert(taker, "1"); } + @Test + public void testTakeEvictedLease() throws LeasingException, InterruptedException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + + builder.withLease("1", "bar").build(); + + // This should not take anything because the lease is new and owned. + builder.takeMutateAssert(taker); + + builder.evictLease("1"); + + // This should take because the lease has been evicted + builder.takeMutateAssert(taker, "1"); + } + /** * Verify that we take leases non-greedily by setting up an environment where there are 4 leases and 2 workers, * only one of which holds a lease. This leaves 3 free leases, but LeaseTaker should decide it needs 2 leases and @@ -203,7 +218,7 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest { /** * Verify that one activity is stolen from the highest loaded server when a server needs more than one lease and no * expired leases are available. Setup: 4 leases, server foo holds 0, bar holds 1, baz holds 5. - * + * * Foo should steal from baz. */ @Test diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java index 00db6a51..ae4b7249 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java @@ -30,6 +30,7 @@ import software.amazon.kinesis.leases.LeaseRenewer; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.LeasingException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; public class TestHarnessBuilder { @@ -96,6 +97,11 @@ public class TestHarnessBuilder { currentTimeNanos += millis * 1000000; } + public void evictLease(String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + Lease lease = leases.get(shardId); + leaseRefresher.evictLease(lease); + } + public Map takeMutateAssert(DynamoDBLeaseTaker taker, int numToTake) throws LeasingException { Map result = taker.takeLeases(timeProvider); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java index f94d82fd..0d4ef538 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java @@ -118,6 +118,7 @@ public class ConsumerStatesTest { private long idleTimeInMillis = 1000L; private Optional logWarningForTaskAfterMillis = Optional.empty(); private SchemaRegistryDecoder schemaRegistryDecoder = null; + private boolean evictLeaseOnShutdown = false; @Before public void setup() { @@ -126,7 +127,7 @@ public class ConsumerStatesTest { taskBackoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, shouldCallProcessRecordsEvenForEmptyRecordList, idleTimeInMillis, INITIAL_POSITION_IN_STREAM, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, shardDetector, - new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory, leaseCleanupManager, schemaRegistryDecoder); + new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory, leaseCleanupManager, schemaRegistryDecoder, evictLeaseOnShutdown); when(shardInfo.shardId()).thenReturn("shardId-000000000000"); when(shardInfo.streamIdentifierSerOpt()).thenReturn(Optional.of(StreamIdentifier.singleStreamInstance(STREAM_NAME).serialize())); consumer = spy(new ShardConsumer(recordsPublisher, executorService, shardInfo, logWarningForTaskAfterMillis, diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java index 46677fb9..01ebf6f2 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java @@ -53,6 +53,10 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import lombok.extern.slf4j.Slf4j; + import org.junit.After; import org.junit.Before; import org.junit.Ignore; @@ -66,15 +70,15 @@ import org.mockito.runners.MockitoJUnitRunner; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; -import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import lombok.extern.slf4j.Slf4j; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCoordinator; +import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardInfo; +import software.amazon.kinesis.lifecycle.ConsumerStates.ShardConsumerState; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.lifecycle.events.TaskExecutionListenerInput; -import software.amazon.kinesis.lifecycle.ConsumerStates.ShardConsumerState; import software.amazon.kinesis.retrieval.RecordsDeliveryAck; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RecordsRetrieved; @@ -136,6 +140,12 @@ public class ShardConsumerTest { private ConsumerState shutdownRequestedAwaitState; @Mock private TaskExecutionListener taskExecutionListener; + @Mock + private LeaseCoordinator leaseCoordinator; + @Mock + private LeaseRefresher leaseRefresher; + @Mock + private Lease lease; private ProcessRecordsInput processRecordsInput; @@ -869,6 +879,186 @@ public class ShardConsumerTest { verifyNoMoreInteractions(taskExecutionListener); } + @Test + public void testLeaseEvictedOnShutdownWhenEnabled() throws Exception { + CyclicBarrier taskBarrier = new CyclicBarrier(2); + + TestPublisher cache = new TestPublisher(); + ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, logWarningForTaskAfterMillis, + shardConsumerArgument, initialState, t -> t, 1, taskExecutionListener, 0); + + mockSuccessfulInitialize(null); + + mockSuccessfulProcessing(taskBarrier); + + when(shardConsumerArgument.evictLeaseOnShutdown()).thenReturn(true); + when(shardConsumerArgument.leaseCoordinator()).thenReturn(leaseCoordinator); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.getCurrentlyHeldLease(ShardInfo.getLeaseKey(shardInfo))).thenReturn(lease); + + when(processingState.shutdownTransition(eq(ShutdownReason.REQUESTED))).thenReturn(shutdownRequestedState); + when(shutdownRequestedState.requiresDataAvailability()).thenReturn(false); + when(shutdownRequestedState.createTask(any(), any(), any())).thenReturn(shutdownRequestedTask); + when(shutdownRequestedState.taskType()).thenReturn(TaskType.SHUTDOWN_NOTIFICATION); + when(shutdownRequestedTask.call()).thenReturn(new TaskResult(null)); + + when(shutdownRequestedState.shutdownTransition(eq(ShutdownReason.REQUESTED))) + .thenReturn(shutdownRequestedAwaitState); + when(shutdownRequestedState.shutdownTransition(eq(ShutdownReason.LEASE_LOST))).thenReturn(shutdownState); + when(shutdownRequestedAwaitState.requiresDataAvailability()).thenReturn(false); + when(shutdownRequestedAwaitState.createTask(any(), any(), any())).thenReturn(null); + when(shutdownRequestedAwaitState.shutdownTransition(eq(ShutdownReason.REQUESTED))) + .thenReturn(shutdownRequestedState); + when(shutdownRequestedAwaitState.shutdownTransition(eq(ShutdownReason.LEASE_LOST))).thenReturn(shutdownState); + when(shutdownRequestedAwaitState.taskType()).thenReturn(TaskType.SHUTDOWN_COMPLETE); + + mockSuccessfulShutdown(null); + + boolean init = consumer.initializeComplete().get(); + while (!init) { + init = consumer.initializeComplete().get(); + } + + consumer.subscribe(); + cache.awaitInitialSetup(); + + cache.publish(); + awaitAndResetBarrier(taskBarrier); + cache.awaitRequest(); + + cache.publish(); + awaitAndResetBarrier(taskBarrier); + cache.awaitRequest(); + + consumer.gracefulShutdown(shutdownNotification); + boolean shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(false)); + shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(false)); + + consumer.leaseLost(); + shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(false)); + shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(true)); + + verify(processingState, times(2)).createTask(any(), any(), any()); + verify(shutdownRequestedState, never()).shutdownTransition(eq(ShutdownReason.LEASE_LOST)); + verify(shutdownRequestedState).createTask(any(), any(), any()); + verify(shutdownRequestedState).shutdownTransition(eq(ShutdownReason.REQUESTED)); + verify(shutdownRequestedAwaitState).createTask(any(), any(), any()); + verify(shutdownRequestedAwaitState).shutdownTransition(eq(ShutdownReason.LEASE_LOST)); + verify(taskExecutionListener, times(1)).beforeTaskExecution(initialTaskInput); + verify(taskExecutionListener, times(2)).beforeTaskExecution(processTaskInput); + verify(taskExecutionListener, times(1)).beforeTaskExecution(shutdownRequestedTaskInput); + verify(taskExecutionListener, times(1)).beforeTaskExecution(shutdownRequestedAwaitTaskInput); + verify(taskExecutionListener, times(1)).beforeTaskExecution(shutdownTaskInput); + + initialTaskInput = initialTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + processTaskInput = processTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + shutdownRequestedTaskInput = shutdownRequestedTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + shutdownTaskInput = shutdownTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + // No task is created/run for this shutdownRequestedAwaitState, so there's no task outcome. + + verify(taskExecutionListener, times(1)).afterTaskExecution(initialTaskInput); + verify(taskExecutionListener, times(2)).afterTaskExecution(processTaskInput); + verify(taskExecutionListener, times(1)).afterTaskExecution(shutdownRequestedTaskInput); + verify(taskExecutionListener, times(1)).afterTaskExecution(shutdownRequestedAwaitTaskInput); + verify(taskExecutionListener, times(1)).afterTaskExecution(shutdownTaskInput); + verifyNoMoreInteractions(taskExecutionListener); + + verify(leaseRefresher).evictLease(lease); + } + + @Test + public void testLeaseNotEvictedOnShutdownByDefault() throws Exception { + CyclicBarrier taskBarrier = new CyclicBarrier(2); + + TestPublisher cache = new TestPublisher(); + ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, logWarningForTaskAfterMillis, + shardConsumerArgument, initialState, t -> t, 1, taskExecutionListener, 0); + + mockSuccessfulInitialize(null); + + mockSuccessfulProcessing(taskBarrier); + + when(shardConsumerArgument.evictLeaseOnShutdown()).thenReturn(false); + + when(processingState.shutdownTransition(eq(ShutdownReason.REQUESTED))).thenReturn(shutdownRequestedState); + when(shutdownRequestedState.requiresDataAvailability()).thenReturn(false); + when(shutdownRequestedState.createTask(any(), any(), any())).thenReturn(shutdownRequestedTask); + when(shutdownRequestedState.taskType()).thenReturn(TaskType.SHUTDOWN_NOTIFICATION); + when(shutdownRequestedTask.call()).thenReturn(new TaskResult(null)); + + when(shutdownRequestedState.shutdownTransition(eq(ShutdownReason.REQUESTED))) + .thenReturn(shutdownRequestedAwaitState); + when(shutdownRequestedState.shutdownTransition(eq(ShutdownReason.LEASE_LOST))).thenReturn(shutdownState); + when(shutdownRequestedAwaitState.requiresDataAvailability()).thenReturn(false); + when(shutdownRequestedAwaitState.createTask(any(), any(), any())).thenReturn(null); + when(shutdownRequestedAwaitState.shutdownTransition(eq(ShutdownReason.REQUESTED))) + .thenReturn(shutdownRequestedState); + when(shutdownRequestedAwaitState.shutdownTransition(eq(ShutdownReason.LEASE_LOST))).thenReturn(shutdownState); + when(shutdownRequestedAwaitState.taskType()).thenReturn(TaskType.SHUTDOWN_COMPLETE); + + mockSuccessfulShutdown(null); + + boolean init = consumer.initializeComplete().get(); + while (!init) { + init = consumer.initializeComplete().get(); + } + + consumer.subscribe(); + cache.awaitInitialSetup(); + + cache.publish(); + awaitAndResetBarrier(taskBarrier); + cache.awaitRequest(); + + cache.publish(); + awaitAndResetBarrier(taskBarrier); + cache.awaitRequest(); + + consumer.gracefulShutdown(shutdownNotification); + boolean shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(false)); + shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(false)); + + consumer.leaseLost(); + shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(false)); + shutdownComplete = consumer.shutdownComplete().get(); + assertThat(shutdownComplete, equalTo(true)); + + verify(processingState, times(2)).createTask(any(), any(), any()); + verify(shutdownRequestedState, never()).shutdownTransition(eq(ShutdownReason.LEASE_LOST)); + verify(shutdownRequestedState).createTask(any(), any(), any()); + verify(shutdownRequestedState).shutdownTransition(eq(ShutdownReason.REQUESTED)); + verify(shutdownRequestedAwaitState).createTask(any(), any(), any()); + verify(shutdownRequestedAwaitState).shutdownTransition(eq(ShutdownReason.LEASE_LOST)); + verify(taskExecutionListener, times(1)).beforeTaskExecution(initialTaskInput); + verify(taskExecutionListener, times(2)).beforeTaskExecution(processTaskInput); + verify(taskExecutionListener, times(1)).beforeTaskExecution(shutdownRequestedTaskInput); + verify(taskExecutionListener, times(1)).beforeTaskExecution(shutdownRequestedAwaitTaskInput); + verify(taskExecutionListener, times(1)).beforeTaskExecution(shutdownTaskInput); + + initialTaskInput = initialTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + processTaskInput = processTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + shutdownRequestedTaskInput = shutdownRequestedTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + shutdownTaskInput = shutdownTaskInput.toBuilder().taskOutcome(TaskOutcome.SUCCESSFUL).build(); + // No task is created/run for this shutdownRequestedAwaitState, so there's no task outcome. + + verify(taskExecutionListener, times(1)).afterTaskExecution(initialTaskInput); + verify(taskExecutionListener, times(2)).afterTaskExecution(processTaskInput); + verify(taskExecutionListener, times(1)).afterTaskExecution(shutdownRequestedTaskInput); + verify(taskExecutionListener, times(1)).afterTaskExecution(shutdownRequestedAwaitTaskInput); + verify(taskExecutionListener, times(1)).afterTaskExecution(shutdownTaskInput); + verifyNoMoreInteractions(taskExecutionListener); + + verifyZeroInteractions(leaseCoordinator); + verifyZeroInteractions(leaseRefresher); + } + private void mockSuccessfulShutdown(CyclicBarrier taskCallBarrier) { mockSuccessfulShutdown(taskCallBarrier, null); } From b1cc48af50c7eb5c39c2dd37e65bca5060c6e48c Mon Sep 17 00:00:00 2001 From: Yu Zeng Date: Fri, 23 Dec 2022 15:34:04 -0800 Subject: [PATCH 02/88] Upgrade aws sdk to the latest version --- .gitignore | 2 +- amazon-kinesis-client-multilang/pom.xml | 4 ++-- amazon-kinesis-client/pom.xml | 2 +- .../software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 4 ++-- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/.gitignore b/.gitignore index 973f5095..7fd632f3 100644 --- a/.gitignore +++ b/.gitignore @@ -2,4 +2,4 @@ target/ AwsCredentials.properties .idea *.iml - +.DS_Store \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index a91a9a6b..e3583af5 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,14 +21,14 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.3 + 2.4.4-SNAPSHOT 4.0.0 amazon-kinesis-client-multilang - 1.12.296 + 1.12.370 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 512daef0..abdd3eb3 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.3 + 2.4.4-SNAPSHOT amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index abbee4da..028c036b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -46,7 +46,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.3"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.4-SNAPSHOT"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index 0e08d6c6..6393737a 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.3 + 2.4.4-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. @@ -33,7 +33,7 @@ - 2.17.268 + 2.19.2 From 780ca178d565d147fe90c01803b3dceeb932eda9 Mon Sep 17 00:00:00 2001 From: Yu Zeng Date: Fri, 23 Dec 2022 15:44:00 -0800 Subject: [PATCH 03/88] Preparation for v2.4.4 --- CHANGELOG.md | 5 +++++ README.md | 7 ++++++- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- 5 files changed, 14 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5eaf5568..66cf7149 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,10 @@ # Changelog +### Release 2.4.4 (December 23, 2022) +* [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk + * aws-java-sdk.version from 1.12.296 -> 1.12.370 + * awssdk.version from 2.17.268 -> 2.19.2 + ### Release 2.4.3 (September 6, 2022) * [#980](https://github.com/awslabs/amazon-kinesis-client/pull/980) logback-classic: 1.2.9 -> 1.4.0 * [#983](https://github.com/awslabs/amazon-kinesis-client/pull/983) diff --git a/README.md b/README.md index ceec72b4..cac15f8e 100644 --- a/README.md +++ b/README.md @@ -50,7 +50,7 @@ The recommended way to use the KCL for Java is to consume it from Maven. software.amazon.kinesis amazon-kinesis-client - 2.4.3 + 2.4.4 ``` @@ -66,6 +66,11 @@ The recommended way to use the KCL for Java is to consume it from Maven. ## Release Notes +### Release 2.4.4 (December 23, 2022) +* [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk + * aws-java-sdk.version from 1.12.296 -> 1.12.370 + * awssdk.version from 2.17.268 -> 2.19.2 + ### Release 2.4.3 (September 6, 2022) * [#980](https://github.com/awslabs/amazon-kinesis-client/pull/980) logback-classic: 1.2.9 -> 1.4.0 * [#983](https://github.com/awslabs/amazon-kinesis-client/pull/983) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index e3583af5..f0766949 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.4-SNAPSHOT + 2.4.4 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index abdd3eb3..474cb033 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.4-SNAPSHOT + 2.4.4 amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 028c036b..9c57b016 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -46,7 +46,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.4-SNAPSHOT"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.4"; /** * Client used to make calls to Kinesis for records retrieval From adb3990481218c9bcd4dbb263b67e4d5b61162a9 Mon Sep 17 00:00:00 2001 From: Yu Zeng Date: Fri, 23 Dec 2022 16:04:01 -0800 Subject: [PATCH 04/88] Correct the KCL version in the main pom --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6393737a..2fd52014 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.4-SNAPSHOT + 2.4.4 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 676bb86b8ef22e031c93f4a5f9204cf497dd88fb Mon Sep 17 00:00:00 2001 From: Yu Zeng Date: Fri, 23 Dec 2022 16:06:15 -0800 Subject: [PATCH 05/88] Preparation for v2.4.4 (Revise CHANGELOG/README) --- CHANGELOG.md | 1 + README.md | 1 + 2 files changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 66cf7149..552bee00 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,7 @@ * [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk * aws-java-sdk.version from 1.12.296 -> 1.12.370 * awssdk.version from 2.17.268 -> 2.19.2 +* [#1020](https://github.com/awslabs/amazon-kinesis-client/pull/1020) Correct the KCL version in the main pom ### Release 2.4.3 (September 6, 2022) * [#980](https://github.com/awslabs/amazon-kinesis-client/pull/980) logback-classic: 1.2.9 -> 1.4.0 diff --git a/README.md b/README.md index cac15f8e..ea2a2b0e 100644 --- a/README.md +++ b/README.md @@ -70,6 +70,7 @@ The recommended way to use the KCL for Java is to consume it from Maven. * [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk * aws-java-sdk.version from 1.12.296 -> 1.12.370 * awssdk.version from 2.17.268 -> 2.19.2 +* [#1020](https://github.com/awslabs/amazon-kinesis-client/pull/1020) Correct the KCL version in the main pom ### Release 2.4.3 (September 6, 2022) * [#980](https://github.com/awslabs/amazon-kinesis-client/pull/980) logback-classic: 1.2.9 -> 1.4.0 From 17d0940f5d076bdf78546c75edd3feb3705e3800 Mon Sep 17 00:00:00 2001 From: Chenyuan Lee <35975040+cylee99@users.noreply.github.com> Date: Tue, 3 Jan 2023 15:10:31 -0800 Subject: [PATCH 06/88] Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request (#1014) * Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request * Update Java docs and minor refactoring * Fix Java doc Co-authored-by: Chenyuan Lee --- .../kinesis/retrieval/IteratorBuilder.java | 35 +++++++++++++++++-- .../retrieval/polling/KinesisDataFetcher.java | 22 +++++++++--- .../polling/PrefetchRecordsPublisher.java | 2 ++ .../retrieval/IteratorBuilderTest.java | 6 ++++ .../polling/KinesisDataFetcherTest.java | 27 ++++++++++++++ 5 files changed, 85 insertions(+), 7 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/IteratorBuilder.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/IteratorBuilder.java index 2b49e031..9e9adf91 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/IteratorBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/IteratorBuilder.java @@ -41,11 +41,42 @@ public class IteratorBuilder { ShardIteratorType.AFTER_SEQUENCE_NUMBER); } + /** + * Creates a GetShardIteratorRequest builder that uses AT_SEQUENCE_NUMBER ShardIteratorType. + * + * @param builder An initial GetShardIteratorRequest builder to be updated. + * @param sequenceNumber The sequence number to restart the request from. + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. + * @return An updated GetShardIteratorRequest.Builder. + */ public static GetShardIteratorRequest.Builder request(GetShardIteratorRequest.Builder builder, - String sequenceNumber, InitialPositionInStreamExtended initialPosition) { + String sequenceNumber, + InitialPositionInStreamExtended initialPosition) { + return getShardIteratorRequest(builder, sequenceNumber, initialPosition, ShardIteratorType.AT_SEQUENCE_NUMBER); + + } + + /** + * Creates a GetShardIteratorRequest builder that uses AFTER_SEQUENCE_NUMBER ShardIteratorType. + * + * @param builder An initial GetShardIteratorRequest builder to be updated. + * @param sequenceNumber The sequence number to restart the request from. + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. + * @return An updated GetShardIteratorRequest.Builder. + */ + public static GetShardIteratorRequest.Builder reconnectRequest(GetShardIteratorRequest.Builder builder, + String sequenceNumber, + InitialPositionInStreamExtended initialPosition) { + return getShardIteratorRequest(builder, sequenceNumber, initialPosition, ShardIteratorType.AFTER_SEQUENCE_NUMBER); + } + + private static GetShardIteratorRequest.Builder getShardIteratorRequest(GetShardIteratorRequest.Builder builder, + String sequenceNumber, + InitialPositionInStreamExtended initialPosition, + ShardIteratorType shardIteratorType) { return apply(builder, GetShardIteratorRequest.Builder::shardIteratorType, GetShardIteratorRequest.Builder::timestamp, GetShardIteratorRequest.Builder::startingSequenceNumber, initialPosition, sequenceNumber, - ShardIteratorType.AT_SEQUENCE_NUMBER); + shardIteratorType); } private final static Map SHARD_ITERATOR_MAPPING; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java index 223ab367..8d36ea8a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java @@ -223,6 +223,12 @@ public class KinesisDataFetcher implements DataFetcher { @Override public void advanceIteratorTo(final String sequenceNumber, final InitialPositionInStreamExtended initialPositionInStream) { + advanceIteratorTo(sequenceNumber, initialPositionInStream, false); + } + + private void advanceIteratorTo(final String sequenceNumber, + final InitialPositionInStreamExtended initialPositionInStream, + boolean isIteratorRestart) { if (sequenceNumber == null) { throw new IllegalArgumentException("SequenceNumber should not be null: shardId " + shardId); } @@ -231,8 +237,12 @@ public class KinesisDataFetcher implements DataFetcher { GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder() .streamName(streamIdentifier.streamName()).shardId(shardId); - GetShardIteratorRequest request = IteratorBuilder.request(builder, sequenceNumber, initialPositionInStream) - .build(); + GetShardIteratorRequest request; + if (isIteratorRestart) { + request = IteratorBuilder.reconnectRequest(builder, sequenceNumber, initialPositionInStream).build(); + } else { + request = IteratorBuilder.request(builder, sequenceNumber, initialPositionInStream).build(); + } // TODO: Check if this metric is fine to be added final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION); @@ -270,8 +280,8 @@ public class KinesisDataFetcher implements DataFetcher { } /** - * Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last - * records call. + * Gets a new next shard iterator from last known sequence number i.e. the sequence number of the last + * record from the last records call. */ @Override public void restartIterator() { @@ -279,7 +289,9 @@ public class KinesisDataFetcher implements DataFetcher { throw new IllegalStateException( "Make sure to initialize the KinesisDataFetcher before restarting the iterator."); } - advanceIteratorTo(lastKnownSequenceNumber, initialPositionInStream); + log.debug("Restarting iterator for sequence number {} on shard id {}", + lastKnownSequenceNumber, streamAndShardId); + advanceIteratorTo(lastKnownSequenceNumber, initialPositionInStream, true); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index ce7e7bb2..1a49cdfb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -502,6 +502,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { calculateHighestSequenceNumber(processRecordsInput), getRecordsResult.nextShardIterator(), PrefetchRecordsRetrieved.generateBatchUniqueIdentifier()); publisherSession.highestSequenceNumber(recordsRetrieved.lastBatchSequenceNumber); + log.debug("Last sequence number retrieved for streamAndShardId {} is {}", streamAndShardId, + recordsRetrieved.lastBatchSequenceNumber); addArrivedRecordsInput(recordsRetrieved); drainQueueForRequests(); } catch (PositionResetException pse) { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/IteratorBuilderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/IteratorBuilderTest.java index 5b04bf8d..db28261e 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/IteratorBuilderTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/IteratorBuilderTest.java @@ -62,6 +62,12 @@ public class IteratorBuilderTest { sequenceNumber(this::gsiBase, this::verifyGsiBase, IteratorBuilder::request, WrappedRequest::wrapped); } + @Test + public void getShardIteratorReconnectTest() { + sequenceNumber(this::gsiBase, this::verifyGsiBase, IteratorBuilder::reconnectRequest, WrappedRequest::wrapped, + ShardIteratorType.AFTER_SEQUENCE_NUMBER); + } + @Test public void subscribeTimestampTest() { timeStampTest(this::stsBase, this::verifyStsBase, IteratorBuilder::request, WrappedRequest::wrapped); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java index 74b0c125..d75f701d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java @@ -435,6 +435,33 @@ public class KinesisDataFetcherTest { assertEquals(restartGetRecordsResponse, kinesisDataFetcher.getRecords().accept()); } + @Test + public void testRestartIteratorUsesAfterSequenceNumberIteratorType() throws Exception { + final String iterator = "iterator"; + final String sequenceNumber = "123"; + + final ArgumentCaptor shardIteratorRequestCaptor = + ArgumentCaptor.forClass(GetShardIteratorRequest.class); + + when(kinesisClient.getShardIterator(shardIteratorRequestCaptor.capture())). + thenReturn(makeGetShardIteratorResonse(iterator)); + + kinesisDataFetcher.initialize(sequenceNumber, INITIAL_POSITION_LATEST); + kinesisDataFetcher.restartIterator(); + // The advanceIteratorTo call should not use AFTER_SEQUENCE_NUMBER iterator + // type unless called by restartIterator + kinesisDataFetcher.advanceIteratorTo(sequenceNumber, INITIAL_POSITION_LATEST); + + final List shardIteratorRequests = shardIteratorRequestCaptor.getAllValues(); + assertEquals(3, shardIteratorRequests.size()); + assertEquals(ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), + shardIteratorRequests.get(0).shardIteratorTypeAsString()); + assertEquals(ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), + shardIteratorRequests.get(1).shardIteratorTypeAsString()); + assertEquals(ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), + shardIteratorRequests.get(2).shardIteratorTypeAsString()); + } + @Test(expected = IllegalStateException.class) public void testRestartIteratorNotInitialized() { kinesisDataFetcher.restartIterator(); From 0e860891232c05a3fa7950fd91aba7ef8b85bf10 Mon Sep 17 00:00:00 2001 From: Chenyuan Lee <35975040+cylee99@users.noreply.github.com> Date: Thu, 5 Jan 2023 08:25:17 -0800 Subject: [PATCH 07/88] Preparation for v2.4.5 (#1021) --- CHANGELOG.md | 3 +++ README.md | 3 +++ amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 6 files changed, 10 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 552bee00..3c7ffd4e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,8 @@ # Changelog +### Release 2.4.5 (January 04, 2023) +* [#1014](https://github.com/awslabs/amazon-kinesis-client/pull/1014) Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request + ### Release 2.4.4 (December 23, 2022) * [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk * aws-java-sdk.version from 1.12.296 -> 1.12.370 diff --git a/README.md b/README.md index ea2a2b0e..160ab682 100644 --- a/README.md +++ b/README.md @@ -66,6 +66,9 @@ The recommended way to use the KCL for Java is to consume it from Maven. ## Release Notes +### Release 2.4.5 (January 04, 2023) +* [#1014](https://github.com/awslabs/amazon-kinesis-client/pull/1014) Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request + ### Release 2.4.4 (December 23, 2022) * [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk * aws-java-sdk.version from 1.12.296 -> 1.12.370 diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index f0766949..41e5e0f9 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.4 + 2.4.5 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 474cb033..77250400 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.4 + 2.4.5 amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 9c57b016..53739e40 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -46,7 +46,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.4"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.5"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index 2fd52014..59feefef 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.4 + 2.4.5 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 4411a3dc772fa853c305002795fa61e9eb4eedbb Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 30 Jan 2023 17:41:11 -0500 Subject: [PATCH 08/88] Added logging w.r.t. `StreamConfig` handling. (#1024) --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../amazon/kinesis/coordinator/Scheduler.java | 77 ++++++++++--------- .../kinesis/retrieval/RetrievalConfig.java | 3 +- pom.xml | 2 +- 5 files changed, 45 insertions(+), 41 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 41e5e0f9..879f7cf9 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.5 + 2.4.6-SNAPSHOT 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 77250400..a80e7ae4 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.5 + 2.4.6-SNAPSHOT amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 5d9f73e9..5c877b80 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -41,7 +41,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; @@ -54,7 +53,6 @@ import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; -import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; @@ -75,7 +73,6 @@ import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer; import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; -import software.amazon.kinesis.leases.exceptions.LeasingException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; @@ -83,7 +80,6 @@ import software.amazon.kinesis.lifecycle.ShardConsumerArgument; import software.amazon.kinesis.lifecycle.ShardConsumerShutdownNotification; import software.amazon.kinesis.lifecycle.ShutdownNotification; import software.amazon.kinesis.lifecycle.ShutdownReason; -import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.CloudWatchMetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; @@ -113,16 +109,16 @@ public class Scheduler implements Runnable { private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1; private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; - private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; + private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1000L; private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; - private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; + private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 60_000L; private static final boolean SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS = false; private static final String MULTI_STREAM_TRACKER = "MultiStreamTracker"; private static final String ACTIVE_STREAMS_COUNT = "ActiveStreams.Count"; private static final String PENDING_STREAMS_DELETION_COUNT = "StreamsPendingDeletion.Count"; private static final String DELETED_STREAMS_COUNT = "DeletedStreams.Count"; - private SchedulerLog slog = new SchedulerLog(); + private final SchedulerLog slog = new SchedulerLog(); private final CheckpointConfig checkpointConfig; private final CoordinatorConfig coordinatorConfig; @@ -175,7 +171,7 @@ public class Scheduler implements Runnable { // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. - private ConcurrentMap shardInfoShardConsumerMap = new ConcurrentHashMap<>(); + private final ConcurrentMap shardInfoShardConsumerMap = new ConcurrentHashMap<>(); private volatile boolean shutdown; private volatile long shutdownStartTimeMillis; @@ -183,7 +179,7 @@ public class Scheduler implements Runnable { private final Object lock = new Object(); - private Stopwatch streamSyncWatch = Stopwatch.createUnstarted(); + private final Stopwatch streamSyncWatch = Stopwatch.createUnstarted(); private boolean leasesSyncedOnAppInit = false; /** @@ -236,8 +232,13 @@ public class Scheduler implements Runnable { return multiStreamTracker.streamConfigList().stream() .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); }, - streamConfig -> - Collections.singletonMap(streamConfig.streamIdentifier(), streamConfig)); + streamConfig -> { + // use a concrete, non-singleton map to allow computeIfAbsent(...) + // without forcing behavioral differences for multi-stream support + final Map map = new HashMap<>(); + map.put(streamConfig.streamIdentifier(), streamConfig); + return map; + }); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); // Determine leaseSerializer based on availability of MultiStreamTracker. @@ -348,7 +349,6 @@ public class Scheduler implements Runnable { log.info("Initializing LeaseCoordinator"); leaseCoordinator.initialize(); - TaskResult result; if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { if (shouldInitiateLeaseSync()) { log.info("Worker {} is initiating the lease sync.", leaseManagementConfig.workerIdentifier()); @@ -450,7 +450,6 @@ public class Scheduler implements Runnable { return leaderDecider.isLeader(leaseManagementConfig.workerIdentifier()); } - /** * Note: This method has package level access solely for testing purposes. * Sync all streams method. @@ -484,10 +483,11 @@ public class Scheduler implements Runnable { // For new streams discovered, do a shard sync and update the currentStreamConfigMap for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { - log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); - ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); + final StreamConfig streamConfig = newStreamConfigMap.get(streamIdentifier); + log.info("Found new stream to process: {}. Syncing shards of that stream.", streamConfig); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(streamConfig); shardSyncTaskManager.submitShardSyncTask(); - currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); + currentStreamConfigMap.put(streamIdentifier, streamConfig); streamsSynced.add(streamIdentifier); } else { if (log.isDebugEnabled()) { @@ -521,12 +521,10 @@ public class Scheduler implements Runnable { // In order to give workers with stale stream info, sufficient time to learn about the new streams // before attempting to delete it, we will be deferring the leases deletion based on the // defer time period. - - currentStreamConfigMap.keySet().stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier)); - + currentStreamConfigMap.keySet().forEach(enqueueStreamLeaseDeletionOperation); } else if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION) { Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiersForLeaseCleanup()).ifPresent( - streamIdentifiers -> streamIdentifiers.stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier))); + streamIdentifiers -> streamIdentifiers.forEach(enqueueStreamLeaseDeletionOperation)); } else { // Remove the old/stale streams identified through the new and existing streams list, without // cleaning up their leases. Disabling deprecated shard sync + lease cleanup through a flag. @@ -614,28 +612,27 @@ public class Scheduler implements Runnable { private Set deleteMultiStreamLeases(Set streamIdentifiers) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + if (streamIdentifiers.isEmpty()) { + return Collections.emptySet(); + } + final Set streamsSynced = new HashSet<>(); - List leases = null; - Map> streamIdToShardsMap = null; - for(StreamIdentifier streamIdentifier : streamIdentifiers) { - if (leases == null) { - // Lazy Load once and use many times for this iteration. - leases = fetchMultiStreamLeases(); - } - if (streamIdToShardsMap == null) { - // Lazy load once and use many times for this iteration. - streamIdToShardsMap = leases.stream().collect(Collectors - .groupingBy(MultiStreamLease::streamIdentifier, - Collectors.toCollection(ArrayList::new))); - } - log.warn("Found old/deleted stream: " + streamIdentifier + ". Directly deleting leases of this stream."); + final List leases = fetchMultiStreamLeases(); + final Map> streamIdToShardsMap = leases.stream().collect( + Collectors.groupingBy(MultiStreamLease::streamIdentifier, Collectors.toCollection(ArrayList::new))); + for (StreamIdentifier streamIdentifier : streamIdentifiers) { // Deleting leases will cause the workers to shutdown the record processors for these shards. if (deleteMultiStreamLeases(streamIdToShardsMap.get(streamIdentifier.serialize()))) { + log.warn("Found old/deleted stream: {}. Directly deleting leases of this stream.", streamIdentifier); currentStreamConfigMap.remove(streamIdentifier); staleStreamDeletionMap.remove(streamIdentifier); streamsSynced.add(streamIdentifier); } } + if (!streamsSynced.isEmpty()) { + // map keys are StreamIdentifiers, which are members of StreamConfig, and therefore redundant + log.info("Streams retained post-deletion: {}", currentStreamConfigMap.values()); + } return streamsSynced; } @@ -655,9 +652,15 @@ public class Scheduler implements Runnable { return true; } - // Generate default StreamConfig for an "orphaned" stream that is in the lease table but not tracked + /** + * Generates default StreamConfig for an "orphaned" stream that is in the lease table but not tracked. + * + * @param streamIdentifier stream for which an orphan config should be generated + */ private StreamConfig getOrphanedStreamConfig(StreamIdentifier streamIdentifier) { - return new StreamConfig(streamIdentifier, orphanedStreamInitialPositionInStream); + final StreamConfig orphanConfig = new StreamConfig(streamIdentifier, orphanedStreamInitialPositionInStream); + log.info("Identified as orphan: {}", orphanConfig); + return orphanConfig; } /** @@ -919,7 +922,7 @@ public class Scheduler implements Runnable { // Irrespective of single stream app or multi stream app, streamConfig should always be available. // If we have a shardInfo, that is not present in currentStreamConfigMap for whatever reason, then return default stream config // to gracefully complete the reading. - final StreamConfig streamConfig = currentStreamConfigMap.getOrDefault(streamIdentifier, getOrphanedStreamConfig(streamIdentifier)); + final StreamConfig streamConfig = currentStreamConfigMap.computeIfAbsent(streamIdentifier, this::getOrphanedStreamConfig); Validate.notNull(streamConfig, "StreamConfig should not be null"); RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, streamConfig, metricsFactory); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 53739e40..abb85612 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -26,6 +26,7 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.KinesisClientLibraryPackage; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.processor.MultiStreamTracker; @@ -46,7 +47,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.5"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.6-SNAPSHOT"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index 59feefef..ac946119 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.5 + 2.4.6-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 6146ff9851ee3a079179ab1415e8ea66fb8167a7 Mon Sep 17 00:00:00 2001 From: CF <6307904+charlesfinley@users.noreply.github.com> Date: Tue, 31 Jan 2023 12:32:10 -0500 Subject: [PATCH 09/88] Typo in Comment (#740) Implemetation -> Implementation --- .../amazon/kinesis/leases/dynamodb/TableCreatorCallback.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/TableCreatorCallback.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/TableCreatorCallback.java index e027908f..23022778 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/TableCreatorCallback.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/TableCreatorCallback.java @@ -21,7 +21,7 @@ package software.amazon.kinesis.leases.dynamodb; @FunctionalInterface public interface TableCreatorCallback { /** - * NoOp implemetation for TableCreatorCallback + * NoOp implementation for TableCreatorCallback */ TableCreatorCallback NOOP_TABLE_CREATOR_CALLBACK = (TableCreatorCallbackInput tableCreatorCallbackInput) -> { // Do nothing From 65c95ed872763b8f0d002ab5dd2c72dae37736e1 Mon Sep 17 00:00:00 2001 From: Jan Sochor <2857409+sochi@users.noreply.github.com> Date: Tue, 31 Jan 2023 18:53:45 +0100 Subject: [PATCH 10/88] Pass isAtShardEnd correctly to processRecords call (#935) The default is false otherwise, i.e., the processor is always getting isAtShardEnd=false. --- .../java/software/amazon/kinesis/lifecycle/ProcessTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java index f05efb91..e4b38815 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java @@ -215,7 +215,7 @@ public class ProcessTask implements ConsumerTask { shardInfoId); final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records).cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime()) - .checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build(); + .isAtShardEnd(input.isAtShardEnd()).checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION); shardInfo.streamIdentifierSerOpt() From 28cb185e9c80d7bc1264ce1f6dffe51b61b5dce3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 31 Jan 2023 14:47:32 -0500 Subject: [PATCH 11/88] Bump protobuf-java from 3.21.5 to 3.21.12 (#1015) Bumps [protobuf-java](https://github.com/protocolbuffers/protobuf) from 3.21.5 to 3.21.12. - [Release notes](https://github.com/protocolbuffers/protobuf/releases) - [Changelog](https://github.com/protocolbuffers/protobuf/blob/main/generate_changelog.py) - [Commits](https://github.com/protocolbuffers/protobuf/compare/v3.21.5...v3.21.12) --- updated-dependencies: - dependency-name: com.google.protobuf:protobuf-java dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index a80e7ae4..21bd9ba8 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -93,7 +93,7 @@ com.google.protobuf protobuf-java - 3.21.5 + 3.21.12 org.apache.commons From 05351933948207d464b9a699c725c30b8c598104 Mon Sep 17 00:00:00 2001 From: Ben Iofel Date: Tue, 31 Jan 2023 18:26:14 -0500 Subject: [PATCH 12/88] Fix warning message typos (#956) Same as #875 --- .../kinesis/lifecycle/ShardConsumerSubscriber.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java index 52ec1468..e8406d92 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java @@ -200,11 +200,11 @@ class ShardConsumerSubscriber implements Subscriber { protected void logOnErrorReadTimeoutWarning(Throwable t) { log.warn("{}: onError(). Cancelling subscription, and marking self as failed. KCL will" - + " recreate the subscription as neccessary to continue processing. If you " - + "are seeing this warning frequently consider increasing the SDK timeouts " - + "by providing an OverrideConfiguration to the kinesis client. Alternatively you" - + "can configure LifecycleConfig.readTimeoutsToIgnoreBeforeWarning to suppress" - + "intermittent ReadTimeout warnings. Last successful request details -- {}", + + " recreate the subscription as necessary to continue processing. If you" + + " are seeing this warning frequently consider increasing the SDK timeouts" + + " by providing an OverrideConfiguration to the kinesis client. Alternatively you" + + " can configure LifecycleConfig.readTimeoutsToIgnoreBeforeWarning to suppress" + + " intermittent ReadTimeout warnings. Last successful request details -- {}", shardInfoId, recordsPublisher.getLastSuccessfulRequestDetails(), t); } From 6cb01001638ff04a6eb9ad8168e4590d825a7599 Mon Sep 17 00:00:00 2001 From: JumpeiAnzai <53547731+JumpeiAnzai@users.noreply.github.com> Date: Wed, 1 Feb 2023 08:36:12 +0900 Subject: [PATCH 13/88] Fixing log message spacing (#795) --- .../amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index 7e8932cf..8404925d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java @@ -381,7 +381,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private void resetRecordsDeliveryStateOnSubscriptionOnInit() { // Clear any lingering records in the queue. if (!recordsDeliveryQueue.isEmpty()) { - log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of" + log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of " + "previous subscription - {}. Last successful request details -- {}", streamAndShardId, subscribeToShardId, lastSuccessfulRequestDetails); recordsDeliveryQueue.clear(); } From dd429a2b1c7bff284b2a9ec9c51c56cb03cd6140 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Wed, 1 Feb 2023 01:39:13 -0500 Subject: [PATCH 14/88] Removed `CHECKSTYLE:OFF` toggles which can invite/obscure sub-par code. (#1027) + removed unused `assertHashRangeOfClosedShardIsCovered(...)` method --- .../leases/HierarchicalShardSyncer.java | 46 ++----------------- 1 file changed, 3 insertions(+), 43 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index 4f677524..19c900d6 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -109,7 +109,6 @@ public class HierarchicalShardSyncer { * @throws ProvisionedThroughputException * @throws KinesisClientLibIOException */ - // CHECKSTYLE:OFF CyclomaticComplexity public synchronized boolean checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final MetricsScope scope, final boolean ignoreUnexpectedChildShards, final boolean isLeaseTableEmpty) @@ -195,38 +194,6 @@ public class HierarchicalShardSyncer { .flatMap(entry -> shardIdToChildShardIdsMap.get(entry.getKey()).stream()).collect(Collectors.toSet()); } - private synchronized void assertHashRangeOfClosedShardIsCovered(final Shard closedShard, - final Map shardIdToShardMap, final Set childShardIds) - throws KinesisClientLibIOException { - BigInteger minStartingHashKeyOfChildren = null; - BigInteger maxEndingHashKeyOfChildren = null; - - final BigInteger startingHashKeyOfClosedShard = new BigInteger(closedShard.hashKeyRange().startingHashKey()); - final BigInteger endingHashKeyOfClosedShard = new BigInteger(closedShard.hashKeyRange().endingHashKey()); - - for (String childShardId : childShardIds) { - final Shard childShard = shardIdToShardMap.get(childShardId); - final BigInteger startingHashKey = new BigInteger(childShard.hashKeyRange().startingHashKey()); - if (minStartingHashKeyOfChildren == null || startingHashKey.compareTo(minStartingHashKeyOfChildren) < 0) { - minStartingHashKeyOfChildren = startingHashKey; - } - - final BigInteger endingHashKey = new BigInteger(childShard.hashKeyRange().endingHashKey()); - if (maxEndingHashKeyOfChildren == null || endingHashKey.compareTo(maxEndingHashKeyOfChildren) > 0) { - maxEndingHashKeyOfChildren = endingHashKey; - } - } - - if (minStartingHashKeyOfChildren == null || maxEndingHashKeyOfChildren == null - || minStartingHashKeyOfChildren.compareTo(startingHashKeyOfClosedShard) > 0 - || maxEndingHashKeyOfChildren.compareTo(endingHashKeyOfClosedShard) < 0) { - throw new KinesisClientLibIOException(String.format( - "Incomplete shard list: hash key range of shard %s is not covered by its child shards.", - closedShard.shardId())); - } - - } - /** * Helper method to construct shardId->setOfChildShardIds map. * Note: This has package access for testing purposes only. @@ -262,7 +229,6 @@ public class HierarchicalShardSyncer { * @return ShardFilter shard filter for the corresponding position in the stream. */ private static ShardFilter getShardFilterFromInitialPosition(InitialPositionInStreamExtended initialPositionInStreamExtended) { - ShardFilter.Builder builder = ShardFilter.builder(); switch (initialPositionInStreamExtended.getInitialPositionInStream()) { @@ -314,7 +280,6 @@ public class HierarchicalShardSyncer { } private static boolean isHashRangeOfShardsComplete(@NonNull List shards) { - if (shards.isEmpty()) { throw new IllegalStateException("No shards found when attempting to validate complete hash range."); } @@ -413,7 +378,6 @@ public class HierarchicalShardSyncer { * @param memoizationContext Memoization of shards that have been evaluated as part of the evaluation * @return true if the shard is a descendant of any current shard (lease already exists) */ - // CHECKSTYLE:OFF CyclomaticComplexity static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, final Map shardIdToShardMapOfAllKinesisShards, @@ -544,8 +508,6 @@ public class HierarchicalShardSyncer { new MultiStreamArgs(false, null)); } - // CHECKSTYLE:ON CyclomaticComplexity - /** * Helper method to get parent shardIds of the current shard - includes the parent shardIds if: * a/ they are not null @@ -750,7 +712,6 @@ public class HierarchicalShardSyncer { return result; } - } @Data @@ -839,11 +800,10 @@ public class HierarchicalShardSyncer { shardIdToNewLeaseMap.put(shardId, lease); } - return new ArrayList(shardIdToNewLeaseMap.values()); + return new ArrayList<>(shardIdToNewLeaseMap.values()); } } - /** * Class to help create leases when the lease table is not initially empty. */ @@ -973,8 +933,8 @@ public class HierarchicalShardSyncer { */ @NoArgsConstructor static class MemoizationContext { - private Map isDescendantMap = new HashMap<>(); - private Map shouldCreateLeaseMap = new HashMap<>(); + private final Map isDescendantMap = new HashMap<>(); + private final Map shouldCreateLeaseMap = new HashMap<>(); Boolean isDescendant(String shardId) { return isDescendantMap.get(shardId); From 17b82a0e6766c0a70d6a52a2f28f2cd59d7ffe75 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Tue, 7 Feb 2023 02:04:34 -0500 Subject: [PATCH 15/88] Refactored `MultiStreamTracker` to provide and enhance OOP for both (#1028) single- and multi-stream trackers. + converted `Scheduler#currentStreamConfigMap` to `ConcurrentHashMap` + eliminated a responsibility from Scheduler (i.e., orphan config generation) --- .../amazon/kinesis/common/ConfigsBuilder.java | 59 ++++++++---- .../kinesis/common/DeprecationUtils.java | 53 +++++++++++ .../amazon/kinesis/coordinator/Scheduler.java | 62 ++++--------- .../kinesis/processor/MultiStreamTracker.java | 41 ++------- .../processor/SingleStreamTracker.java | 81 +++++++++++++++++ .../kinesis/processor/StreamTracker.java | 85 ++++++++++++++++++ .../kinesis/retrieval/RetrievalConfig.java | 46 +++++++--- .../kinesis/common/ConfigsBuilderTest.java | 89 +++++++++++++++++++ .../kinesis/common/DeprecationUtilsTest.java | 45 ++++++++++ .../processor/SingleStreamTrackerTest.java | 66 ++++++++++++++ .../retrieval/RetrievalConfigTest.java | 85 ++++++++++++++++++ 11 files changed, 605 insertions(+), 107 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DeprecationUtils.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/StreamTracker.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/DeprecationUtilsTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/processor/SingleStreamTrackerTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java index 09d28495..a5bbfebe 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java @@ -15,6 +15,8 @@ package software.amazon.kinesis.common; +import java.util.function.Function; + import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.Setter; @@ -35,6 +37,8 @@ import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.MultiStreamTracker; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.processor.StreamTracker; import software.amazon.kinesis.retrieval.RetrievalConfig; /** @@ -46,9 +50,18 @@ public class ConfigsBuilder { /** * Either the name of the stream to consume records from * Or MultiStreamTracker for all the streams to consume records from + * + * @deprecated Both single- and multi-stream support is now provided by {@link StreamTracker}. + * @see #streamTracker */ + @Deprecated private Either appStreamTracker; + /** + * Stream(s) to be consumed by this KCL application. + */ + private StreamTracker streamTracker; + /** * Application name for the KCL Worker */ @@ -115,7 +128,8 @@ public class ConfigsBuilder { } /** - * Constructor to initialize ConfigsBuilder with StreamName + * Constructor to initialize ConfigsBuilder for a single stream. + * * @param streamName * @param applicationName * @param kinesisClient @@ -128,18 +142,19 @@ public class ConfigsBuilder { @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { - this.appStreamTracker = Either.right(streamName); - this.applicationName = applicationName; - this.kinesisClient = kinesisClient; - this.dynamoDBClient = dynamoDBClient; - this.cloudWatchClient = cloudWatchClient; - this.workerIdentifier = workerIdentifier; - this.shardRecordProcessorFactory = shardRecordProcessorFactory; + this(new SingleStreamTracker(streamName), + applicationName, + kinesisClient, + dynamoDBClient, + cloudWatchClient, + workerIdentifier, + shardRecordProcessorFactory); } /** - * Constructor to initialize ConfigsBuilder with MultiStreamTracker - * @param multiStreamTracker + * Constructor to initialize ConfigsBuilder + * + * @param streamTracker tracker for single- or multi-stream processing * @param applicationName * @param kinesisClient * @param dynamoDBClient @@ -147,17 +162,30 @@ public class ConfigsBuilder { * @param workerIdentifier * @param shardRecordProcessorFactory */ - public ConfigsBuilder(@NonNull MultiStreamTracker multiStreamTracker, @NonNull String applicationName, + public ConfigsBuilder(@NonNull StreamTracker streamTracker, @NonNull String applicationName, @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { - this.appStreamTracker = Either.left(multiStreamTracker); this.applicationName = applicationName; this.kinesisClient = kinesisClient; this.dynamoDBClient = dynamoDBClient; this.cloudWatchClient = cloudWatchClient; this.workerIdentifier = workerIdentifier; this.shardRecordProcessorFactory = shardRecordProcessorFactory; + + // construct both streamTracker and appStreamTracker + streamTracker(streamTracker); + } + + public void appStreamTracker(Either appStreamTracker) { + this.appStreamTracker = appStreamTracker; + streamTracker = appStreamTracker.map(Function.identity(), SingleStreamTracker::new); + } + + public void streamTracker(StreamTracker streamTracker) { + this.streamTracker = streamTracker; + this.appStreamTracker = DeprecationUtils.convert(streamTracker, + singleStreamTracker -> singleStreamTracker.streamConfigList().get(0).streamIdentifier().streamName()); } /** @@ -205,7 +233,6 @@ public class ConfigsBuilder { return new MetricsConfig(cloudWatchClient(), namespace()); } - /** * Creates a new instance of ProcessorConfig * @@ -221,10 +248,6 @@ public class ConfigsBuilder { * @return RetrievalConfig */ public RetrievalConfig retrievalConfig() { - final RetrievalConfig retrievalConfig = - appStreamTracker.map( - multiStreamTracker -> new RetrievalConfig(kinesisClient(), multiStreamTracker, applicationName()), - streamName -> new RetrievalConfig(kinesisClient(), streamName, applicationName())); - return retrievalConfig; + return new RetrievalConfig(kinesisClient(), streamTracker(), applicationName()); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DeprecationUtils.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DeprecationUtils.java new file mode 100644 index 00000000..5d8782e0 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DeprecationUtils.java @@ -0,0 +1,53 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import java.util.function.Function; + +import software.amazon.awssdk.utils.Either; +import software.amazon.kinesis.processor.MultiStreamTracker; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.processor.StreamTracker; + +/** + * Utility methods to facilitate deprecated code until that deprecated code + * can be safely removed. + */ +public final class DeprecationUtils { + + private DeprecationUtils() { + throw new UnsupportedOperationException("utility class"); + } + + /** + * Converts a {@link StreamTracker} into the deprecated {@code Either} convention. + * + * @param streamTracker tracker to convert + */ + @Deprecated + public static Either convert( + StreamTracker streamTracker, + Function converter) { + if (streamTracker instanceof MultiStreamTracker) { + return Either.left((MultiStreamTracker) streamTracker); + } else if (streamTracker instanceof SingleStreamTracker) { + return Either.right(converter.apply((SingleStreamTracker) streamTracker)); + } else { + throw new IllegalArgumentException("Unhandled StreamTracker: " + streamTracker); + } + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 5c877b80..8bc4fc98 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -53,7 +53,6 @@ import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; -import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.HierarchicalShardSyncer; @@ -88,10 +87,10 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.processor.Checkpointer; import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; -import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShutdownNotificationAware; +import software.amazon.kinesis.processor.StreamTracker; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; @@ -138,7 +137,6 @@ public class Scheduler implements Runnable { private final ExecutorService executorService; private final DiagnosticEventFactory diagnosticEventFactory; private final DiagnosticEventHandler diagnosticEventHandler; - // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; private final Function shardSyncTaskManagerProvider; private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); @@ -152,10 +150,9 @@ public class Scheduler implements Runnable { private final long failoverTimeMillis; private final long taskBackoffTimeMillis; private final boolean isMultiStreamMode; - private final Map currentStreamConfigMap; - private MultiStreamTracker multiStreamTracker; - private FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy; - private InitialPositionInStreamExtended orphanedStreamInitialPositionInStream; + private final Map currentStreamConfigMap = new ConcurrentHashMap<>(); + private final StreamTracker streamTracker; + private final FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; @@ -222,23 +219,12 @@ public class Scheduler implements Runnable { this.retrievalConfig = retrievalConfig; this.applicationName = this.coordinatorConfig.applicationName(); - this.isMultiStreamMode = this.retrievalConfig.appStreamTracker().map( - multiStreamTracker -> true, streamConfig -> false); - this.currentStreamConfigMap = this.retrievalConfig.appStreamTracker().map( - multiStreamTracker -> { - this.multiStreamTracker = multiStreamTracker; - this.formerStreamsLeasesDeletionStrategy = multiStreamTracker.formerStreamsLeasesDeletionStrategy(); - this.orphanedStreamInitialPositionInStream = multiStreamTracker.orphanedStreamInitialPositionInStream(); - return multiStreamTracker.streamConfigList().stream() - .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); - }, - streamConfig -> { - // use a concrete, non-singleton map to allow computeIfAbsent(...) - // without forcing behavioral differences for multi-stream support - final Map map = new HashMap<>(); - map.put(streamConfig.streamIdentifier(), streamConfig); - return map; - }); + this.streamTracker = retrievalConfig.streamTracker(); + this.isMultiStreamMode = streamTracker.isMultiStream(); + this.formerStreamsLeasesDeletionStrategy = streamTracker.formerStreamsLeasesDeletionStrategy(); + streamTracker.streamConfigList().forEach( + sc -> currentStreamConfigMap.put(sc.streamIdentifier(), sc)); + this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); // Determine leaseSerializer based on availability of MultiStreamTracker. @@ -464,12 +450,8 @@ public class Scheduler implements Runnable { final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, MULTI_STREAM_TRACKER); try { - - final Map newStreamConfigMap = new HashMap<>(); - final Duration waitPeriodToDeleteOldStreams = formerStreamsLeasesDeletionStrategy.waitPeriodToDeleteFormerStreams(); - // Making an immutable copy - newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() - .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); + final Map newStreamConfigMap = streamTracker.streamConfigList() + .stream().collect(Collectors.toMap(StreamConfig::streamIdentifier, Function.identity())); List leases; @@ -549,6 +531,8 @@ public class Scheduler implements Runnable { } } + final Duration waitPeriodToDeleteOldStreams = + formerStreamsLeasesDeletionStrategy.waitPeriodToDeleteFormerStreams(); // Now let's scan the streamIdentifiersForLeaseCleanup eligible for deferred deletion and delete them. // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and // the streamIdentifiersForLeaseCleanup are not present in the latest snapshot. @@ -594,7 +578,7 @@ public class Scheduler implements Runnable { .collect(Collectors.toSet()); for (StreamIdentifier streamIdentifier : streamIdentifiers) { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { - currentStreamConfigMap.put(streamIdentifier, getOrphanedStreamConfig(streamIdentifier)); + currentStreamConfigMap.put(streamIdentifier, streamTracker.createStreamConfig(streamIdentifier)); } } } @@ -652,17 +636,6 @@ public class Scheduler implements Runnable { return true; } - /** - * Generates default StreamConfig for an "orphaned" stream that is in the lease table but not tracked. - * - * @param streamIdentifier stream for which an orphan config should be generated - */ - private StreamConfig getOrphanedStreamConfig(StreamIdentifier streamIdentifier) { - final StreamConfig orphanConfig = new StreamConfig(streamIdentifier, orphanedStreamInitialPositionInStream); - log.info("Identified as orphan: {}", orphanConfig); - return orphanConfig; - } - /** * Returns whether worker can shutdown immediately. Note that this method is called from Worker's {{@link #run()} * method before every loop run, so method must do minimum amount of work to not impact shard processing timings. @@ -922,7 +895,10 @@ public class Scheduler implements Runnable { // Irrespective of single stream app or multi stream app, streamConfig should always be available. // If we have a shardInfo, that is not present in currentStreamConfigMap for whatever reason, then return default stream config // to gracefully complete the reading. - final StreamConfig streamConfig = currentStreamConfigMap.computeIfAbsent(streamIdentifier, this::getOrphanedStreamConfig); + StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); + if (streamConfig == null) { + streamConfig = streamTracker.createStreamConfig(streamIdentifier); + } Validate.notNull(streamConfig, "StreamConfig should not be null"); RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, streamConfig, metricsFactory); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java index 7e878e2c..ead38333 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java @@ -15,43 +15,14 @@ package software.amazon.kinesis.processor; -import software.amazon.kinesis.common.InitialPositionInStream; -import software.amazon.kinesis.common.InitialPositionInStreamExtended; -import software.amazon.kinesis.common.StreamConfig; - -import java.util.List; - /** - * Interface for stream trackers. This is useful for KCL Workers that need - * to consume data from multiple streams. - * KCL will periodically probe this interface to learn about the new and old streams. + * Tracker for consuming multiple Kinesis streams. */ -public interface MultiStreamTracker { +public interface MultiStreamTracker extends StreamTracker { - /** - * Returns the list of stream config, to be processed by the current application. - * Note that the streams list CAN be changed during the application runtime. - * This method will be called periodically by the KCL to learn about the change in streams to process. - * - * @return List of StreamConfig - */ - List streamConfigList(); - - /** - * Strategy to delete leases of old streams in the lease table. - * Note that the strategy CANNOT be changed during the application runtime. - * - * @return StreamsLeasesDeletionStrategy - */ - FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy(); - - /** - * The position for getting records from an "orphaned" stream that is in the lease table but not tracked - * Default assumes that the stream no longer need to be tracked, so use LATEST for faster shard end. - * - *

Default value: {@link InitialPositionInStream#LATEST}

- */ - default InitialPositionInStreamExtended orphanedStreamInitialPositionInStream() { - return InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST); + @Override + default boolean isMultiStream() { + return true; } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java new file mode 100644 index 00000000..703c4881 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java @@ -0,0 +1,81 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.processor; + +import java.util.Collections; +import java.util.List; + +import lombok.EqualsAndHashCode; +import lombok.NonNull; +import lombok.ToString; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; + +/** + * Tracker for consuming a single Kinesis stream. + */ +@EqualsAndHashCode +@ToString +public class SingleStreamTracker implements StreamTracker { + + /** + * By default, single-stream applications should expect the target stream + * to exist for the duration of the application. Therefore, there is no + * expectation for the leases to be deleted mid-execution. + */ + private static final FormerStreamsLeasesDeletionStrategy NO_LEASE_DELETION = + new FormerStreamsLeasesDeletionStrategy.NoLeaseDeletionStrategy(); + + private final StreamIdentifier streamIdentifier; + + private final List streamConfigs; + + public SingleStreamTracker(String streamName) { + this(StreamIdentifier.singleStreamInstance(streamName)); + } + + public SingleStreamTracker(StreamIdentifier streamIdentifier) { + this(streamIdentifier, DEFAULT_POSITION_IN_STREAM); + } + + public SingleStreamTracker( + StreamIdentifier streamIdentifier, + @NonNull InitialPositionInStreamExtended initialPosition) { + this(streamIdentifier, new StreamConfig(streamIdentifier, initialPosition)); + } + + public SingleStreamTracker(@NonNull StreamIdentifier streamIdentifier, @NonNull StreamConfig streamConfig) { + this.streamIdentifier = streamIdentifier; + this.streamConfigs = Collections.singletonList(streamConfig); + } + + @Override + public List streamConfigList() { + return streamConfigs; + } + + @Override + public FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy() { + return NO_LEASE_DELETION; + } + + @Override + public boolean isMultiStream() { + return false; + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/StreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/StreamTracker.java new file mode 100644 index 00000000..befa3709 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/StreamTracker.java @@ -0,0 +1,85 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.processor; + +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; + +import java.util.List; + +/** + * Interface for stream trackers. + * KCL will periodically probe this interface to learn about the new and old streams. + */ +public interface StreamTracker { + + /** + * Default position to begin consuming records from a Kinesis stream. + * + * @see #orphanedStreamInitialPositionInStream() + */ + InitialPositionInStreamExtended DEFAULT_POSITION_IN_STREAM = + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST); + + /** + * Returns the list of stream config, to be processed by the current application. + * Note that the streams list CAN be changed during the application runtime. + * This method will be called periodically by the KCL to learn about the change in streams to process. + * + * @return List of StreamConfig + */ + List streamConfigList(); + + /** + * Strategy to delete leases of old streams in the lease table. + * Note that the strategy CANNOT be changed during the application runtime. + * + * @return StreamsLeasesDeletionStrategy + */ + FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy(); + + /** + * The position for getting records from an "orphaned" stream that is in the lease table but not tracked + * Default assumes that the stream no longer need to be tracked, so use LATEST for faster shard end. + * + *

Default value: {@link InitialPositionInStream#LATEST}

+ */ + default InitialPositionInStreamExtended orphanedStreamInitialPositionInStream() { + return DEFAULT_POSITION_IN_STREAM; + } + + /** + * Returns a new {@link StreamConfig} for the provided stream identifier. + * + * @param streamIdentifier stream for which to create a new config + */ + default StreamConfig createStreamConfig(StreamIdentifier streamIdentifier) { + return new StreamConfig(streamIdentifier, orphanedStreamInitialPositionInStream()); + } + + /** + * Returns true if this application should accommodate the consumption of + * more than one Kinesis stream. + *

+ * This method must be consistent. Varying the returned value will + * have indeterminate, and likely problematic, effects on stream processing. + *

+ */ + boolean isMultiStream(); + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index abb85612..000b71b7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -24,12 +24,14 @@ import lombok.ToString; import lombok.experimental.Accessors; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.utils.Either; +import software.amazon.kinesis.common.DeprecationUtils; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; -import software.amazon.kinesis.common.KinesisClientLibraryPackage; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.processor.MultiStreamTracker; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.processor.StreamTracker; import software.amazon.kinesis.retrieval.fanout.FanOutConfig; import software.amazon.kinesis.retrieval.polling.PollingConfig; @@ -67,9 +69,17 @@ public class RetrievalConfig { /** * AppStreamTracker either for multi stream tracking or single stream + * + * @deprecated Both single- and multi-stream support is now provided by {@link StreamTracker}. + * @see #streamTracker */ private Either appStreamTracker; + /** + * Stream(s) to be consumed by this KCL application. + */ + private StreamTracker streamTracker; + /** * Backoff time between consecutive ListShards calls. * @@ -95,7 +105,12 @@ public class RetrievalConfig { *

* Default value: {@link InitialPositionInStream#LATEST} *

+ * + * @deprecated Initial stream position is now handled by {@link StreamTracker}. + * @see StreamTracker#orphanedStreamInitialPositionInStream() + * @see StreamTracker#createConfig(StreamIdentifier) */ + @Deprecated private InitialPositionInStreamExtended initialPositionInStreamExtended = InitialPositionInStreamExtended .newInitialPosition(InitialPositionInStream.LATEST); @@ -105,27 +120,36 @@ public class RetrievalConfig { public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName, @NonNull String applicationName) { - this.kinesisClient = kinesisAsyncClient; - this.appStreamTracker = Either - .right(new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStreamExtended)); - this.applicationName = applicationName; + this(kinesisAsyncClient, new SingleStreamTracker(streamName), applicationName); } - public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull MultiStreamTracker multiStreamTracker, + public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull StreamTracker streamTracker, @NonNull String applicationName) { this.kinesisClient = kinesisAsyncClient; - this.appStreamTracker = Either.left(multiStreamTracker); + this.streamTracker = streamTracker; this.applicationName = applicationName; + this.appStreamTracker = DeprecationUtils.convert(streamTracker, + singleStreamTracker -> singleStreamTracker.streamConfigList().get(0)); } + /** + * + * @param initialPositionInStreamExtended + * + * @deprecated Initial stream position is now handled by {@link StreamTracker}. + * @see StreamTracker#orphanedStreamInitialPositionInStream() + * @see StreamTracker#createConfig(StreamIdentifier) + */ + @Deprecated public RetrievalConfig initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) { - final StreamConfig[] streamConfig = new StreamConfig[1]; this.appStreamTracker.apply(multiStreamTracker -> { throw new IllegalArgumentException( "Cannot set initialPositionInStreamExtended when multiStreamTracker is set"); - }, sc -> streamConfig[0] = sc); - this.appStreamTracker = Either - .right(new StreamConfig(streamConfig[0].streamIdentifier(), initialPositionInStreamExtended)); + }, sc -> { + final StreamConfig updatedConfig = new StreamConfig(sc.streamIdentifier(), initialPositionInStreamExtended); + streamTracker = new SingleStreamTracker(sc.streamIdentifier(), updatedConfig); + appStreamTracker = Either.right(updatedConfig); + }); return this; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java new file mode 100644 index 00000000..8ea8f818 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java @@ -0,0 +1,89 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import java.util.Arrays; +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.mockito.Mockito.mock; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.processor.MultiStreamTracker; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.processor.StreamTracker; + +public class ConfigsBuilderTest { + + @Mock + private KinesisAsyncClient mockKinesisClient; + + @Mock + private DynamoDbAsyncClient mockDynamoClient; + + @Mock + private CloudWatchAsyncClient mockCloudWatchClient; + + @Mock + private ShardRecordProcessorFactory mockShardProcessorFactory; + + private static final String APPLICATION_NAME = ConfigsBuilderTest.class.getSimpleName(); + private static final String WORKER_IDENTIFIER = "worker-id"; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testTrackerConstruction() { + final String streamName = "single-stream"; + final ConfigsBuilder configByName = createConfig(streamName); + final ConfigsBuilder configBySingleTracker = createConfig(new SingleStreamTracker(streamName)); + + for (final ConfigsBuilder cb : Arrays.asList(configByName, configBySingleTracker)) { + assertEquals(Optional.empty(), cb.appStreamTracker().left()); + assertEquals(streamName, cb.appStreamTracker().right().get()); + assertEquals(streamName, cb.streamTracker().streamConfigList().get(0).streamIdentifier().streamName()); + assertFalse(cb.streamTracker().isMultiStream()); + } + + final StreamTracker mockMultiStreamTracker = mock(MultiStreamTracker.class); + final ConfigsBuilder configByMultiTracker = createConfig(mockMultiStreamTracker); + assertEquals(Optional.empty(), configByMultiTracker.appStreamTracker().right()); + assertEquals(mockMultiStreamTracker, configByMultiTracker.appStreamTracker().left().get()); + assertEquals(mockMultiStreamTracker, configByMultiTracker.streamTracker()); + } + + private ConfigsBuilder createConfig(String streamName) { + return new ConfigsBuilder(streamName, APPLICATION_NAME, mockKinesisClient, mockDynamoClient, + mockCloudWatchClient, WORKER_IDENTIFIER, mockShardProcessorFactory); + } + + private ConfigsBuilder createConfig(StreamTracker streamTracker) { + return new ConfigsBuilder(streamTracker, APPLICATION_NAME, mockKinesisClient, mockDynamoClient, + mockCloudWatchClient, WORKER_IDENTIFIER, mockShardProcessorFactory); + } + +} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/DeprecationUtilsTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/DeprecationUtilsTest.java new file mode 100644 index 00000000..39991b78 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/DeprecationUtilsTest.java @@ -0,0 +1,45 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import java.util.function.Function; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mock; + +import org.junit.Test; +import software.amazon.awssdk.utils.Either; +import software.amazon.kinesis.processor.MultiStreamTracker; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.processor.StreamTracker; + +public class DeprecationUtilsTest { + + @Test + public void testTrackerConversion() { + final StreamTracker mockMultiTracker = mock(MultiStreamTracker.class); + assertEquals(Either.left(mockMultiTracker), DeprecationUtils.convert(mockMultiTracker, Function.identity())); + + final StreamTracker mockSingleTracker = mock(SingleStreamTracker.class); + assertEquals(Either.right(mockSingleTracker), DeprecationUtils.convert(mockSingleTracker, Function.identity())); + } + + @Test(expected = IllegalArgumentException.class) + public void testUnsupportedStreamTrackerConversion() { + DeprecationUtils.convert(mock(StreamTracker.class), Function.identity()); + } + +} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/processor/SingleStreamTrackerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/processor/SingleStreamTrackerTest.java new file mode 100644 index 00000000..9ae19ba3 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/processor/SingleStreamTrackerTest.java @@ -0,0 +1,66 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.processor; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertThat; + +import org.hamcrest.Matchers; +import org.junit.Test; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; + +public class SingleStreamTrackerTest { + + private static final String STREAM_NAME = SingleStreamTrackerTest.class.getSimpleName(); + + @Test + public void testDefaults() { + validate(new SingleStreamTracker(STREAM_NAME)); + validate(new SingleStreamTracker(StreamIdentifier.singleStreamInstance(STREAM_NAME))); + } + + @Test + public void testInitialPositionConstructor() { + final InitialPositionInStreamExtended expectedPosition = + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); + assertNotEquals(expectedPosition, StreamTracker.DEFAULT_POSITION_IN_STREAM); + + final StreamTracker tracker = new SingleStreamTracker( + StreamIdentifier.singleStreamInstance(STREAM_NAME), expectedPosition); + validate(tracker, expectedPosition); + } + + private static void validate(StreamTracker tracker) { + validate(tracker, StreamTracker.DEFAULT_POSITION_IN_STREAM); + } + + private static void validate(StreamTracker tracker, InitialPositionInStreamExtended expectedPosition) { + assertEquals(1, tracker.streamConfigList().size()); + assertFalse(tracker.isMultiStream()); + assertThat(tracker.formerStreamsLeasesDeletionStrategy(), + Matchers.instanceOf(FormerStreamsLeasesDeletionStrategy.NoLeaseDeletionStrategy.class)); + + final StreamConfig config = tracker.streamConfigList().get(0); + assertEquals(STREAM_NAME, config.streamIdentifier().streamName()); + assertEquals(expectedPosition, config.initialPositionInStreamExtended()); + } + +} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java new file mode 100644 index 00000000..041ac71e --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java @@ -0,0 +1,85 @@ +package software.amazon.kinesis.retrieval; + +import java.util.Arrays; +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.mockito.Mockito.mock; +import static software.amazon.kinesis.common.InitialPositionInStream.LATEST; +import static software.amazon.kinesis.common.InitialPositionInStream.TRIM_HORIZON; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.processor.MultiStreamTracker; +import software.amazon.kinesis.processor.SingleStreamTracker; +import software.amazon.kinesis.processor.StreamTracker; + +public class RetrievalConfigTest { + + private static final String APPLICATION_NAME = RetrievalConfigTest.class.getSimpleName(); + + @Mock + private KinesisAsyncClient mockKinesisClient; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testTrackerConstruction() { + final String streamName = "single-stream"; + final RetrievalConfig configByName = createConfig(streamName); + final SingleStreamTracker singleTracker = new SingleStreamTracker(streamName); + final RetrievalConfig configBySingleTracker = createConfig(singleTracker); + + for (final RetrievalConfig rc : Arrays.asList(configByName, configBySingleTracker)) { + assertEquals(Optional.empty(), rc.appStreamTracker().left()); + assertEquals(singleTracker, rc.streamTracker()); + assertEquals(1, rc.streamTracker().streamConfigList().size()); + assertFalse(rc.streamTracker().isMultiStream()); + } + + final StreamTracker mockMultiStreamTracker = mock(MultiStreamTracker.class); + final RetrievalConfig configByMultiTracker = createConfig(mockMultiStreamTracker); + assertEquals(Optional.empty(), configByMultiTracker.appStreamTracker().right()); + assertEquals(mockMultiStreamTracker, configByMultiTracker.appStreamTracker().left().get()); + assertEquals(mockMultiStreamTracker, configByMultiTracker.streamTracker()); + } + + @Test + public void testUpdateInitialPositionInSingleStream() { + final RetrievalConfig config = createConfig(new SingleStreamTracker("foo")); + + for (final StreamConfig sc : config.streamTracker().streamConfigList()) { + assertEquals(LATEST, sc.initialPositionInStreamExtended().getInitialPositionInStream()); + } + config.initialPositionInStreamExtended( + InitialPositionInStreamExtended.newInitialPosition(TRIM_HORIZON)); + for (final StreamConfig sc : config.streamTracker().streamConfigList()) { + assertEquals(TRIM_HORIZON, sc.initialPositionInStreamExtended().getInitialPositionInStream()); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testUpdateInitialPositionInMultiStream() { + final RetrievalConfig config = createConfig(mock(MultiStreamTracker.class)); + config.initialPositionInStreamExtended( + InitialPositionInStreamExtended.newInitialPosition(TRIM_HORIZON)); + } + + private RetrievalConfig createConfig(String streamName) { + return new RetrievalConfig(mockKinesisClient, streamName, APPLICATION_NAME); + } + + private RetrievalConfig createConfig(StreamTracker streamTracker) { + return new RetrievalConfig(mockKinesisClient, streamTracker, APPLICATION_NAME); + } + +} \ No newline at end of file From 5715e944e2c941521f7d514ce97c5111bc9155a3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Feb 2023 12:18:53 -0500 Subject: [PATCH 16/88] Bump slf4j-api from 2.0.0 to 2.0.6 (#1029) Bumps [slf4j-api](https://github.com/qos-ch/slf4j) from 2.0.0 to 2.0.6. - [Release notes](https://github.com/qos-ch/slf4j/releases) - [Commits](https://github.com/qos-ch/slf4j/compare/v_2.0.0...v_2.0.6) --- updated-dependencies: - dependency-name: org.slf4j:slf4j-api dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 21bd9ba8..3a130e99 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -50,7 +50,7 @@ 1.0.392 libsqlite4java ${project.build.directory}/test-lib - 2.0.0 + 2.0.6 1.1.13 From af71d9e224bd29334ea949c838292f29ac96c087 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Feb 2023 12:19:15 -0500 Subject: [PATCH 17/88] Bump awssdk.version from 2.19.2 to 2.19.31 (#1030) Bumps `awssdk.version` from 2.19.2 to 2.19.31. Updates `kinesis` from 2.19.2 to 2.19.31 Updates `dynamodb` from 2.19.2 to 2.19.31 Updates `cloudwatch` from 2.19.2 to 2.19.31 Updates `netty-nio-client` from 2.19.2 to 2.19.31 Updates `sts` from 2.19.2 to 2.19.31 --- updated-dependencies: - dependency-name: software.amazon.awssdk:kinesis dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:dynamodb dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:cloudwatch dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:netty-nio-client dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:sts dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index ac946119..8475338f 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.19.2 + 2.19.31 From 34f19c5a7b9207ea9ca3225cba828af22fe65750 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 7 Feb 2023 12:20:31 -0500 Subject: [PATCH 18/88] Bump rxjava from 3.1.5 to 3.1.6 (#1032) Bumps [rxjava](https://github.com/ReactiveX/RxJava) from 3.1.5 to 3.1.6. - [Release notes](https://github.com/ReactiveX/RxJava/releases) - [Commits](https://github.com/ReactiveX/RxJava/compare/v3.1.5...v3.1.6) --- updated-dependencies: - dependency-name: io.reactivex.rxjava3:rxjava dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 3a130e99..19119ed0 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -109,7 +109,7 @@ io.reactivex.rxjava3 rxjava - 3.1.5 + 3.1.6 From 9fb58a22bf7a2b97e8a27849639cfdd37d690699 Mon Sep 17 00:00:00 2001 From: pelaezryan Date: Mon, 13 Feb 2023 09:28:54 -0800 Subject: [PATCH 19/88] Increased logging verbosity around lease management. Also included additional javadocs for methods (#1040) Co-authored-by: Ryan Pelaez --- .../coordinator/PeriodicShardSyncManager.java | 31 ++++++++++++ .../amazon/kinesis/coordinator/Scheduler.java | 3 +- .../leases/HierarchicalShardSyncer.java | 47 +++++++++++++------ .../kinesis/leases/LeaseCleanupManager.java | 17 +++++++ .../dynamodb/DynamoDBLeaseRefresher.java | 20 ++++++-- .../leases/dynamodb/DynamoDBLeaseRenewer.java | 7 +-- .../leases/dynamodb/DynamoDBLeaseTaker.java | 3 +- .../kinesis/lifecycle/ShutdownTask.java | 5 +- 8 files changed, 106 insertions(+), 27 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java index b4999bec..a2d05e6d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java @@ -202,6 +202,10 @@ class PeriodicShardSyncManager { log.warn( "Failed to submit shard sync task for stream {}. This could be due to the previous pending shard sync task.", shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + } else { + log.info("Submitted shard sync task for stream {} because of reason {}", + shardSyncTaskManager.shardDetector().streamIdentifier().streamName(), + shardSyncResponse.reasonForDecision()); } } else { log.info("Skipping shard sync for {} due to the reason - {}", streamConfigEntry.getKey(), @@ -222,6 +226,14 @@ class PeriodicShardSyncManager { } } + /** + * Retrieve all the streams, along with their associated leases + * @param streamIdentifiersToFilter + * @return + * @throws DependencyException + * @throws ProvisionedThroughputException + * @throws InvalidStateException + */ private Map> getStreamToLeasesMap( final Set streamIdentifiersToFilter) throws DependencyException, ProvisionedThroughputException, InvalidStateException { @@ -242,6 +254,13 @@ class PeriodicShardSyncManager { } } + + /** + * Given a list of leases for a stream, determine if a shard sync is necessary. + * @param streamIdentifier + * @param leases + * @return + */ @VisibleForTesting ShardSyncResponse checkForShardSync(StreamIdentifier streamIdentifier, List leases) { if (CollectionUtils.isNullOrEmpty(leases)) { @@ -272,12 +291,24 @@ class PeriodicShardSyncManager { } } + /** + * Object containing metadata about the state of a shard sync + */ @Value @Accessors(fluent = true) @VisibleForTesting static class ShardSyncResponse { + + /** + * Flag to determine if a shard sync is necessary or not + */ private final boolean shouldDoShardSync; + private final boolean isHoleDetected; + + /** + * Reason behind the state of 'shouldDoShardSync' flag + */ private final String reasonForDecision; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 8bc4fc98..743be28a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -331,8 +331,7 @@ public class Scheduler implements Runnable { for (int i = 0; (!isDone) && (i < maxInitializationAttempts); i++) { try { - log.info("Initialization attempt {}", (i + 1)); - log.info("Initializing LeaseCoordinator"); + log.info("Initializing LeaseCoordinator attempt {}", (i + 1)); leaseCoordinator.initialize(); if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index 19c900d6..068db578 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -148,17 +148,24 @@ public class HierarchicalShardSyncer { new NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); final List newLeasesToCreate = determineNewLeasesToCreate(leaseSynchronizer, latestShards, currentLeases, initialPosition, inconsistentShardIds, multiStreamArgs); - log.debug("{} - Num new leases to create: {}", streamIdentifier, newLeasesToCreate.size()); + log.info("{} - Number of new leases to create: {}", streamIdentifier, newLeasesToCreate.size()); + + final Set createdLeases = new HashSet<>(); + for (Lease lease : newLeasesToCreate) { long startTime = System.currentTimeMillis(); boolean success = false; try { - leaseRefresher.createLeaseIfNotExists(lease); + if(leaseRefresher.createLeaseIfNotExists(lease)) { + createdLeases.add(lease); + } success = true; - } finally { + } + finally { MetricsUtil.addSuccessAndLatency(scope, "CreateLease", success, startTime, MetricsLevel.DETAILED); } } + log.info("{} - Newly created leases {}: {}", streamIdentifier, createdLeases.size(), createdLeases); final List trackedLeases = new ArrayList<>(currentLeases); trackedLeases.addAll(newLeasesToCreate); return true; @@ -398,6 +405,7 @@ public class HierarchicalShardSyncer { isDescendant = true; // We don't need to add leases of its ancestors, // because we'd have done it when creating a lease for this shard. + log.debug("{} - Shard {} is a descendant shard of an existing shard. Skipping lease creation", streamIdentifier, shardId); } else { final Shard shard = shardIdToShardMapOfAllKinesisShards.get(shardId); @@ -474,9 +482,12 @@ public class HierarchicalShardSyncer { if (descendantParentShardIds.contains(parentShardId) && !initialPosition.getInitialPositionInStream() .equals(InitialPositionInStream.AT_TIMESTAMP)) { + log.info("Setting Lease '{}' checkpoint to 'TRIM_HORIZON'. Checkpoint was previously set to {}", lease.leaseKey(), lease.checkpoint()); lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); } else { - lease.checkpoint(convertToCheckpoint(initialPosition)); + final ExtendedSequenceNumber newCheckpoint = convertToCheckpoint(initialPosition); + log.info("Setting Lease '{}' checkpoint to '{}'. Checkpoint was previously set to {}", lease.leaseKey(), newCheckpoint, lease.checkpoint()); + lease.checkpoint(newCheckpoint); } } } @@ -512,7 +523,7 @@ public class HierarchicalShardSyncer { * Helper method to get parent shardIds of the current shard - includes the parent shardIds if: * a/ they are not null * b/ if they exist in the current shard map (i.e. haven't expired) - * + * * @param shard Will return parents of this shard * @param shardIdToShardMapOfAllKinesisShards ShardId->Shard map containing all shards obtained via DescribeStream. * @return Set of parentShardIds @@ -538,6 +549,12 @@ public class HierarchicalShardSyncer { : newKCLLeaseForChildShard(childShard); } + /** + * Generate a lease object for the given Child Shard. Checkpoint is set to TRIM_HORIZON + * @param childShard Shard for which a lease should be created + * @return Lease for the shard + * @throws InvalidStateException If the child shard has no parent shards + */ private static Lease newKCLLeaseForChildShard(final ChildShard childShard) throws InvalidStateException { Lease newLease = new Lease(); newLease.leaseKey(childShard.shardId()); @@ -571,7 +588,7 @@ public class HierarchicalShardSyncer { /** * Helper method to create a new Lease POJO for a shard. * Note: Package level access only for testing purposes - * + * * @param shard * @return */ @@ -611,7 +628,7 @@ public class HierarchicalShardSyncer { /** * Helper method to construct a shardId->Shard map for the specified list of shards. - * + * * @param shards List of shards * @return ShardId->Shard map */ @@ -622,7 +639,7 @@ public class HierarchicalShardSyncer { /** * Helper method to return all the open shards for a stream. * Note: Package level access only for testing purposes. - * + * * @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. */ @@ -633,7 +650,7 @@ public class HierarchicalShardSyncer { private static ExtendedSequenceNumber convertToCheckpoint(final InitialPositionInStreamExtended position) { ExtendedSequenceNumber checkpoint = null; - + if (position.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)) { checkpoint = ExtendedSequenceNumber.TRIM_HORIZON; } else if (position.getInitialPositionInStream().equals(InitialPositionInStream.LATEST)) { @@ -641,7 +658,7 @@ public class HierarchicalShardSyncer { } else if (position.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { checkpoint = ExtendedSequenceNumber.AT_TIMESTAMP; } - + return checkpoint; } @@ -688,7 +705,7 @@ public class HierarchicalShardSyncer { * We assume that lease1 and lease2 are: * a/ not null, * b/ shards (if found) have non-null starting sequence numbers - * + * * {@inheritDoc} */ @Override @@ -698,18 +715,18 @@ public class HierarchicalShardSyncer { final String shardId2 = shardIdFromLeaseDeducer.apply(lease2, multiStreamArgs); final Shard shard1 = shardIdToShardMap.get(shardId1); final Shard shard2 = shardIdToShardMap.get(shardId2); - + // If we found shards for the two leases, use comparison of the starting sequence numbers if (shard1 != null && shard2 != null) { BigInteger sequenceNumber1 = new BigInteger(shard1.sequenceNumberRange().startingSequenceNumber()); BigInteger sequenceNumber2 = new BigInteger(shard2.sequenceNumberRange().startingSequenceNumber()); - result = sequenceNumber1.compareTo(sequenceNumber2); + result = sequenceNumber1.compareTo(sequenceNumber2); } - + if (result == 0) { result = shardId1.compareTo(shardId2); } - + return result; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java index a5928c2a..f9e52e1c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java @@ -145,10 +145,18 @@ public class LeaseCleanupManager { return deletionQueue.size(); } + /** + * + * @return true if the 'Completed Lease Stopwatch' has elapsed more time than the 'Completed Lease Cleanup Interval' + */ private boolean timeToCheckForCompletedShard() { return completedLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= completedLeaseCleanupIntervalMillis; } + /** + * + * @return true if the 'Garbage Lease Stopwatch' has elapsed more time than the 'Garbage Lease Cleanup Interval' + */ private boolean timeToCheckForGarbageShard() { return garbageLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= garbageLeaseCleanupIntervalMillis; } @@ -230,6 +238,15 @@ public class LeaseCleanupManager { return true; } + /** + * Check if the all of the parent shards for a given lease have an ongoing lease. If any one parent still has a lease, return false. Otherwise return true + * @param lease + * @param shardInfo + * @return + * @throws DependencyException + * @throws ProvisionedThroughputException + * @throws InvalidStateException + */ private boolean allParentShardLeasesDeleted(Lease lease, ShardInfo shardInfo) throws DependencyException, ProvisionedThroughputException, InvalidStateException { for (String parentShard : lease.parentShardIds()) { final Lease parentLease = leaseCoordinator.leaseRefresher().getLease(ShardInfo.getLeaseKey(shardInfo, parentShard)); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java index 361db9f9..acb61a38 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java @@ -169,7 +169,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { request = CreateTableRequest.builder().tableName(table).keySchema(serializer.getKeySchema()) .attributeDefinitions(serializer.getAttributeDefinitions()) .billingMode(billingMode).build(); - }else{ + } else { request = CreateTableRequest.builder().tableName(table).keySchema(serializer.getKeySchema()) .attributeDefinitions(serializer.getAttributeDefinitions()).provisionedThroughput(throughput) .build(); @@ -429,7 +429,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean createLeaseIfNotExists(@NonNull final Lease lease) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - log.debug("Creating lease {}", lease); + log.debug("Creating lease: {}", lease); PutItemRequest request = PutItemRequest.builder().tableName(table).item(serializer.toDynamoRecord(lease)) .expected(serializer.getDynamoNonexistantExpectation()).build(); @@ -452,6 +452,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } catch (DynamoDbException | TimeoutException e) { throw convertAndRethrowExceptions("create", lease.leaseKey(), e); } + log.info("Created lease: {}",lease); return true; } @@ -476,7 +477,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { return null; } else { final Lease lease = serializer.fromDynamoRecord(dynamoRecord); - log.debug("Got lease {}", lease); + log.debug("Retrieved lease: {}", lease); return lease; } } catch (ExecutionException e) { @@ -535,6 +536,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } lease.leaseCounter(lease.leaseCounter() + 1); + log.debug("Renewed lease with key {}", lease.leaseKey()); return true; } @@ -582,6 +584,8 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { lease.ownerSwitchesSinceCheckpoint(lease.ownerSwitchesSinceCheckpoint() + 1); } + log.info("Transferred lease {} ownership from {} to {}", lease.leaseKey(), oldOwner, owner); + return true; } @@ -620,6 +624,8 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { lease.leaseOwner(null); lease.leaseCounter(lease.leaseCounter() + 1); + + log.info("Evicted lease with leaseKey {}", lease.leaseKey()); return true; } @@ -648,6 +654,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } catch (DynamoDbException | TimeoutException e) { throw convertAndRethrowExceptions("deleteAll", lease.leaseKey(), e); } + log.debug("Deleted lease {} from table {}", lease.leaseKey(), table); } } @@ -675,6 +682,8 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } catch (DynamoDbException | TimeoutException e) { throw convertAndRethrowExceptions("delete", lease.leaseKey(), e); } + + log.info("Deleted lease with leaseKey {}", lease.leaseKey()); } /** @@ -683,7 +692,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean updateLease(@NonNull final Lease lease) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - log.debug("Updating lease {}", lease); + log.debug("Updating lease: {}", lease); final AWSExceptionManager exceptionManager = createExceptionManager(); exceptionManager.add(ConditionalCheckFailedException.class, t -> t); @@ -711,6 +720,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } lease.leaseCounter(lease.leaseCounter() + 1); + log.info("Updated lease {}.", lease.leaseKey()); return true; } @@ -738,6 +748,8 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } catch (DynamoDbException | TimeoutException e) { throw convertAndRethrowExceptions("update", lease.leaseKey(), e); } + + log.info("Updated lease without expectation {}.", lease); } /** diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java index e457b5ec..ab2d38c5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java @@ -242,7 +242,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { /** * Internal method to return a lease with a specific lease key only if we currently hold it. - * + * * @param leaseKey key of lease to return * @param now current timestamp for old-ness checking * @return non-authoritative copy of the held lease, or null if we don't currently hold it @@ -309,6 +309,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { long startTime = System.currentTimeMillis(); boolean success = false; try { + log.info("Updating lease from {} to {}", authoritativeLease, lease); synchronized (authoritativeLease) { authoritativeLease.update(lease); boolean updatedLease = leaseRefresher.updateLease(authoritativeLease); @@ -325,7 +326,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { /* * Remove only if the value currently in the map is the same as the authoritative lease. We're * guarding against a pause after the concurrency token check above. It plays out like so: - * + * * 1) Concurrency token check passes * 2) Pause. Lose lease, re-acquire lease. This requires at least one lease counter update. * 3) Unpause. leaseRefresher.updateLease fails conditional write due to counter updates, returns @@ -333,7 +334,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { * 4) ownedLeases.remove(key, value) doesn't do anything because authoritativeLease does not * .equals() the re-acquired version in the map on the basis of lease counter. This is what we want. * If we just used ownedLease.remove(key), we would have pro-actively removed a lease incorrectly. - * + * * Note that there is a subtlety here - Lease.equals() deliberately does not check the concurrency * token, but it does check the lease counter, so this scheme works. */ diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java index a90ef56e..4a4f086f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java @@ -259,6 +259,7 @@ public class DynamoDBLeaseTaker implements LeaseTaker { leasesToTake = leasesToTake.stream().map(lease -> { if (lease.isMarkedForLeaseSteal()) { try { + log.debug("Updating stale lease {}.", lease.leaseKey()); return leaseRefresher.getLease(lease.leaseKey()); } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { log.warn("Failed to fetch latest state of the lease {} that needs to be stolen, " @@ -408,7 +409,7 @@ public class DynamoDBLeaseTaker implements LeaseTaker { target = 1; } else { /* - * numWorkers must be < numLeases. + * if we have made it here, it means there are more leases than workers * * Our target for each worker is numLeases / numWorkers (+1 if numWorkers doesn't evenly divide numLeases) */ diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index c2c5c790..c436f38a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -267,13 +267,14 @@ public class ShutdownTask implements ConsumerTask { } } } - // Attempt create leases for child shards. for(ChildShard childShard : childShards) { final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId()); if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) { + log.debug("{} - Shard {} - Attempting to create lease for child shard {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseKey); final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); - log.info("Shard {}: Created child shard lease: {}", shardInfo.shardId(), leaseToCreate.leaseKey()); + + log.info("{} - Shard {}: Created child shard lease: {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseToCreate); } } } From 4d94efac8f416a019547501fe58a1e06ccca1d0d Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 13 Feb 2023 13:16:28 -0500 Subject: [PATCH 20/88] Optimization: 9~15% improvement in `KinesisDataFetcher` wall-time after (#1034) converting `AWSExceptionManger` to a static variable. --- .../kinesis/leases/KinesisShardDetector.java | 26 +++++++++---- .../retrieval/AWSExceptionManager.java | 4 +- .../retrieval/polling/KinesisDataFetcher.java | 22 +++++------ .../polling/KinesisDataFetcherTest.java | 37 ++++++++----------- 4 files changed, 47 insertions(+), 42 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java index 189ba18b..0c3de1bd 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -64,6 +64,22 @@ import software.amazon.kinesis.retrieval.AWSExceptionManager; @KinesisClientInternalApi public class KinesisShardDetector implements ShardDetector { + /** + * Reusable {@link AWSExceptionManager}. + *

+ * N.B. This instance is mutable, but thread-safe for read-only use. + *

+ */ + private static final AWSExceptionManager AWS_EXCEPTION_MANAGER; + + static { + AWS_EXCEPTION_MANAGER = new AWSExceptionManager(); + AWS_EXCEPTION_MANAGER.add(KinesisException.class, t -> t); + AWS_EXCEPTION_MANAGER.add(LimitExceededException.class, t -> t); + AWS_EXCEPTION_MANAGER.add(ResourceInUseException.class, t -> t); + AWS_EXCEPTION_MANAGER.add(ResourceNotFoundException.class, t -> t); + } + @NonNull private final KinesisAsyncClient kinesisClient; @NonNull @Getter @@ -78,7 +94,7 @@ public class KinesisShardDetector implements ShardDetector { private volatile Map cachedShardMap = null; private volatile Instant lastCacheUpdateTime; @Getter(AccessLevel.PACKAGE) - private AtomicInteger cacheMisses = new AtomicInteger(0); + private final AtomicInteger cacheMisses = new AtomicInteger(0); @Deprecated public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, @@ -186,12 +202,6 @@ public class KinesisShardDetector implements ShardDetector { } private ListShardsResponse listShards(ShardFilter shardFilter, final String nextToken) { - final AWSExceptionManager exceptionManager = new AWSExceptionManager(); - exceptionManager.add(ResourceNotFoundException.class, t -> t); - exceptionManager.add(LimitExceededException.class, t -> t); - exceptionManager.add(ResourceInUseException.class, t -> t); - exceptionManager.add(KinesisException.class, t -> t); - ListShardsRequest.Builder builder = KinesisRequestsBuilder.listShardsRequestBuilder(); if (StringUtils.isEmpty(nextToken)) { builder = builder.streamName(streamIdentifier.streamName()).shardFilter(shardFilter); @@ -211,7 +221,7 @@ public class KinesisShardDetector implements ShardDetector { try { result = getListShardsResponse(request); } catch (ExecutionException e) { - throw exceptionManager.apply(e.getCause()); + throw AWS_EXCEPTION_MANAGER.apply(e.getCause()); } catch (InterruptedException e) { // TODO: check if this is the correct behavior for Interrupted Exception log.debug("Interrupted exception caught, shutdown initiated, returning null"); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/AWSExceptionManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/AWSExceptionManager.java index 65d5d0c0..8081b946 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/AWSExceptionManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/AWSExceptionManager.java @@ -26,7 +26,9 @@ import lombok.experimental.Accessors; import software.amazon.kinesis.annotations.KinesisClientInternalApi; /** - * + * Traverses a {@code Throwable} class inheritance in search of a mapping + * function which will convert that throwable into a {@code RuntimeException}. + * If no mapping function is found, the default function will be applied. */ @KinesisClientInternalApi public class AWSExceptionManager { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java index 8d36ea8a..d17828e9 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java @@ -28,14 +28,12 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse; import software.amazon.awssdk.services.kinesis.model.KinesisException; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; -import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -48,7 +46,6 @@ import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.retrieval.AWSExceptionManager; import software.amazon.kinesis.retrieval.DataFetcherProviderConfig; import software.amazon.kinesis.retrieval.DataFetcherResult; -import software.amazon.kinesis.retrieval.DataRetrievalUtil; import software.amazon.kinesis.retrieval.IteratorBuilder; import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RetryableRetrievalException; @@ -66,6 +63,14 @@ public class KinesisDataFetcher implements DataFetcher { private static final String METRICS_PREFIX = "KinesisDataFetcher"; private static final String OPERATION = "ProcessTask"; + /** + * Reusable {@link AWSExceptionManager}. + *

+ * N.B. This instance is mutable, but thread-safe for read-only use. + *

+ */ + private static final AWSExceptionManager AWS_EXCEPTION_MANAGER = createExceptionManager(); + @NonNull private final KinesisAsyncClient kinesisClient; @NonNull @Getter @@ -91,8 +96,6 @@ public class KinesisDataFetcher implements DataFetcher { /** * Note: This method has package level access for testing purposes. - * - * @return nextIterator */ @Getter(AccessLevel.PACKAGE) private String nextIterator; @@ -233,8 +236,6 @@ public class KinesisDataFetcher implements DataFetcher { throw new IllegalArgumentException("SequenceNumber should not be null: shardId " + shardId); } - final AWSExceptionManager exceptionManager = createExceptionManager(); - GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder() .streamName(streamIdentifier.streamName()).shardId(shardId); GetShardIteratorRequest request; @@ -256,7 +257,7 @@ public class KinesisDataFetcher implements DataFetcher { nextIterator = getNextIterator(request); success = true; } catch (ExecutionException e) { - throw exceptionManager.apply(e.getCause()); + throw AWS_EXCEPTION_MANAGER.apply(e.getCause()); } catch (InterruptedException e) { // TODO: Check behavior throw new RuntimeException(e); @@ -328,7 +329,6 @@ public class KinesisDataFetcher implements DataFetcher { @Override public GetRecordsResponse getRecords(@NonNull final String nextIterator) { - final AWSExceptionManager exceptionManager = createExceptionManager(); GetRecordsRequest request = getGetRecordsRequest(nextIterator); final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION); @@ -341,7 +341,7 @@ public class KinesisDataFetcher implements DataFetcher { success = true; return response; } catch (ExecutionException e) { - throw exceptionManager.apply(e.getCause()); + throw AWS_EXCEPTION_MANAGER.apply(e.getCause()); } catch (InterruptedException e) { // TODO: Check behavior log.debug("{} : Interrupt called on method, shutdown initiated", streamAndShardId); @@ -355,7 +355,7 @@ public class KinesisDataFetcher implements DataFetcher { } } - private AWSExceptionManager createExceptionManager() { + private static AWSExceptionManager createExceptionManager() { final AWSExceptionManager exceptionManager = new AWSExceptionManager(); exceptionManager.add(ResourceNotFoundException.class, t -> t); exceptionManager.add(KinesisException.class, t -> t); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java index d75f701d..2e09f34a 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; @@ -154,14 +153,13 @@ public class KinesisDataFetcherTest { testInitializeAndFetch("foo", null, INITIAL_POSITION_LATEST); } - private CompletableFuture makeGetShardIteratorResonse(String shardIterator) - throws InterruptedException, ExecutionException { + private CompletableFuture makeGetShardIteratorResponse(String shardIterator) { return CompletableFuture .completedFuture(GetShardIteratorResponse.builder().shardIterator(shardIterator).build()); } @Test - public void testadvanceIteratorTo() throws KinesisClientLibException, InterruptedException, ExecutionException { + public void testAdvanceIteratorTo() throws KinesisClientLibException { final Checkpointer checkpoint = mock(Checkpointer.class); final String iteratorA = "foo"; final String iteratorB = "bar"; @@ -172,8 +170,9 @@ public class KinesisDataFetcherTest { .forClass(GetShardIteratorRequest.class); when(kinesisClient.getShardIterator(shardIteratorRequestCaptor.capture())) - .thenReturn(makeGetShardIteratorResonse(iteratorA)).thenReturn(makeGetShardIteratorResonse(iteratorA)) - .thenReturn(makeGetShardIteratorResonse(iteratorB)); + .thenReturn(makeGetShardIteratorResponse(iteratorA)) + .thenReturn(makeGetShardIteratorResponse(iteratorA)) + .thenReturn(makeGetShardIteratorResponse(iteratorB)); when(checkpoint.getCheckpoint(SHARD_ID)).thenReturn(new ExtendedSequenceNumber(seqA)); kinesisDataFetcher.initialize(seqA, null); @@ -203,7 +202,7 @@ public class KinesisDataFetcherTest { } @Test - public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() throws InterruptedException, ExecutionException { + public void testAdvanceIteratorToTrimHorizonLatestAndAtTimestamp(){ final ArgumentCaptor requestCaptor = ArgumentCaptor .forClass(GetShardIteratorRequest.class); final String iteratorHorizon = "TRIM_HORIZON"; @@ -218,9 +217,9 @@ public class KinesisDataFetcherTest { tsReq.toBuilder().timestamp(INITIAL_POSITION_AT_TIMESTAMP.getTimestamp().toInstant()).build()); when(kinesisClient.getShardIterator(requestCaptor.capture())) - .thenReturn(makeGetShardIteratorResonse(iteratorHorizon)) - .thenReturn(makeGetShardIteratorResonse(iteratorLatest)) - .thenReturn(makeGetShardIteratorResonse(iteratorAtTimestamp)); + .thenReturn(makeGetShardIteratorResponse(iteratorHorizon)) + .thenReturn(makeGetShardIteratorResponse(iteratorLatest)) + .thenReturn(makeGetShardIteratorResponse(iteratorAtTimestamp)); kinesisDataFetcher.advanceIteratorTo(ShardIteratorType.TRIM_HORIZON.toString(), INITIAL_POSITION_TRIM_HORIZON); assertEquals(iteratorHorizon, kinesisDataFetcher.getNextIterator()); @@ -261,7 +260,7 @@ public class KinesisDataFetcherTest { // Set up proxy mock methods when(kinesisClient.getShardIterator(iteratorCaptor.capture())) - .thenReturn(makeGetShardIteratorResonse(nextIterator)); + .thenReturn(makeGetShardIteratorResponse(nextIterator)); when(kinesisClient.getRecords(recordsCaptor.capture())).thenReturn(future); when(future.get(anyLong(), any(TimeUnit.class))).thenThrow( new ExecutionException(ResourceNotFoundException.builder().message("Test Exception").build())); @@ -302,7 +301,6 @@ public class KinesisDataFetcherTest { // Call records of dataFetcher which will throw an exception getRecordsRetrievalStrategy.getRecords(MAX_RECORDS); - } @Test @@ -318,7 +316,7 @@ public class KinesisDataFetcherTest { final CompletableFuture future = mock(CompletableFuture.class); when(kinesisClient.getShardIterator(iteratorCaptor.capture())) - .thenReturn(makeGetShardIteratorResonse(nextIterator)); + .thenReturn(makeGetShardIteratorResponse(nextIterator)); when(kinesisClient.getRecords(recordsCaptor.capture())).thenReturn(future); when(future.get(anyLong(), any(TimeUnit.class))).thenThrow( new ExecutionException(ResourceNotFoundException.builder().message("Test Exception").build())); @@ -331,8 +329,7 @@ public class KinesisDataFetcherTest { assertEquals(expectedRecordsRequest.shardIterator(), recordsCaptor.getValue().shardIterator()); } - private CompletableFuture makeGetRecordsResponse(String nextIterator, List records) - throws InterruptedException, ExecutionException { + private CompletableFuture makeGetRecordsResponse(String nextIterator, List records) { List childShards = new ArrayList<>(); if(nextIterator == null) { childShards = createChildShards(); @@ -368,7 +365,6 @@ public class KinesisDataFetcherTest { final String initialIterator = "InitialIterator"; final String nextIterator1 = "NextIteratorOne"; final String nextIterator2 = "NextIteratorTwo"; - final String nextIterator3 = "NextIteratorThree"; final CompletableFuture nonAdvancingResult1 = makeGetRecordsResponse(initialIterator, null); final CompletableFuture nonAdvancingResult2 = makeGetRecordsResponse(nextIterator1, null); final CompletableFuture finalNonAdvancingResult = makeGetRecordsResponse(nextIterator2, @@ -378,7 +374,7 @@ public class KinesisDataFetcherTest { final CompletableFuture finalAdvancingResult = makeGetRecordsResponse(null, null); when(kinesisClient.getShardIterator(iteratorCaptor.capture())) - .thenReturn(makeGetShardIteratorResonse(initialIterator)); + .thenReturn(makeGetShardIteratorResponse(initialIterator)); when(kinesisClient.getRecords(recordsCaptor.capture())).thenReturn(nonAdvancingResult1, advancingResult1, nonAdvancingResult2, advancingResult2, finalNonAdvancingResult, finalAdvancingResult); @@ -397,8 +393,6 @@ public class KinesisDataFetcherTest { assertAdvanced(finalAdvancingResult.get(), nextIterator2, null); verify(kinesisClient, times(6)).getRecords(any(GetRecordsRequest.class)); - - reset(kinesisClient); DataFetcherResult terminal = kinesisDataFetcher.getRecords(); @@ -444,7 +438,7 @@ public class KinesisDataFetcherTest { ArgumentCaptor.forClass(GetShardIteratorRequest.class); when(kinesisClient.getShardIterator(shardIteratorRequestCaptor.capture())). - thenReturn(makeGetShardIteratorResonse(iterator)); + thenReturn(makeGetShardIteratorResponse(iterator)); kinesisDataFetcher.initialize(sequenceNumber, INITIAL_POSITION_LATEST); kinesisDataFetcher.restartIterator(); @@ -547,10 +541,9 @@ public class KinesisDataFetcherTest { } else if (iteratorType.equals(ShardIteratorType.AT_SEQUENCE_NUMBER.toString())) { expectedIteratorRequest = expectedIteratorRequest.toBuilder().startingSequenceNumber(seqNo).build(); } - final GetRecordsRequest expectedRecordsRequest = makeGetRecordsRequest(iterator); when(kinesisClient.getShardIterator(iteratorCaptor.capture())) - .thenReturn(makeGetShardIteratorResonse(iterator)); + .thenReturn(makeGetShardIteratorResponse(iterator)); when(kinesisClient.getRecords(recordsCaptor.capture())) .thenReturn(makeGetRecordsResponse(null, expectedRecords)); From 5bfd1ab28973c317cda1ba046f3c7e9381594509 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 13 Feb 2023 13:20:54 -0500 Subject: [PATCH 21/88] Release Note updates to avoid duplication and bitrot (e.g., 1.x release (#1035) notes are not ported forward to 2.x). --- CHANGELOG.md | 295 +-------------------------------------------------- README.md | 165 +--------------------------- 2 files changed, 8 insertions(+), 452 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3c7ffd4e..33391211 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,9 @@ # Changelog +For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md) + +--- + ### Release 2.4.5 (January 04, 2023) * [#1014](https://github.com/awslabs/amazon-kinesis-client/pull/1014) Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request @@ -481,297 +485,6 @@ Suppression can be configured by setting `LifecycleConfig#readTimeoutsToIgnoreBe * MultiLangDaemon is now a separate module The MultiLangDaemon has been separated to its own Maven module and is no longer available in `amazon-kinesis-client`. To include the MultiLangDaemon, add a dependency on `amazon-kinesis-client-multilang`. -## Release 1.9.1 (April 30, 2018) -* Added the ability to create a prepared checkpoint when at `SHARD_END`. - * [PR #301](https://github.com/awslabs/amazon-kinesis-client/pull/301) -* Added the ability to subscribe to worker state change events. - * [PR #291](https://github.com/awslabs/amazon-kinesis-client/pull/291) -* Added support for custom lease managers. - A custom `LeaseManager` can be provided to `Worker.Builder` that will be used to provide lease services. - This makes it possible to implement custom lease management systems in addition to the default DynamoDB system. - * [PR #297](https://github.com/awslabs/amazon-kinesis-client/pull/297) -* Updated the version of the AWS Java SDK to 1.11.219 - -## Release 1.9.0 (February 6, 2018) -* Introducing support for ListShards API. This API is used in place of DescribeStream API to provide more throughput during ShardSyncTask. Please consult the [AWS Documentation for ListShards](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ListShards.html) for more information. - * ListShards supports higher call rate, which should reduce instances of throttling when attempting to synchronize the shard list. - * __WARNING: `ListShards` is a new API, and may require updating any explicit IAM policies__ - * Added configuration parameters for ListShards usage - - | Name | Default | Description | - | ---- | ------- | ----------- | - | [listShardsBackoffTimeInMillis](https://github.com/awslabs/amazon-kinesis-client/blob/3ae916c5fcdccd6b835c86ba7f6f53dd5b4c8b04/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1412) | 1500 ms | This is the default backoff time between 2 ListShards calls when throttled. | - | [listShardsRetryAttempts](https://github.com/awslabs/amazon-kinesis-client/blob/3ae916c5fcdccd6b835c86ba7f6f53dd5b4c8b04/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1423) | 50 | This is the maximum number of times the KinesisProxy will retry to make ListShards calls on being throttled. | - -* Updating the version of AWS Java SDK to 1.11.272. - * Version 1.11.272 is now the minimum support version of the SDK. -* Deprecating the following methods, and classes. These methods, and classes will be removed in a future release. - * Deprecated [IKinesisProxy#getStreamInfo](https://github.com/awslabs/amazon-kinesis-client/blob/3ae916c5fcdccd6b835c86ba7f6f53dd5b4c8b04/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java#L48-L62). - * Deprecated [IKinesisProxyFactory](https://github.com/awslabs/amazon-kinesis-client/blob/3ae916c5fcdccd6b835c86ba7f6f53dd5b4c8b04/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxyFactory.java). - * Deprecated [KinesisProxyFactory](https://github.com/awslabs/amazon-kinesis-client/blob/3ae916c5fcdccd6b835c86ba7f6f53dd5b4c8b04/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxyFactory.java). - * Deprecated certain [KinesisProxy](https://github.com/awslabs/amazon-kinesis-client/blob/3ae916c5fcdccd6b835c86ba7f6f53dd5b4c8b04/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java) constructors. - * [PR #293](https://github.com/awslabs/amazon-kinesis-client/pull/293) - -## Release 1.8.10 -* Allow providing a custom IKinesisProxy implementation. - * [PR #274](https://github.com/awslabs/amazon-kinesis-client/pull/274) -* Checkpointing on a different thread should no longer emit a warning about NullMetricsScope. - * [PR #284](https://github.com/awslabs/amazon-kinesis-client/pull/284) - * [Issue #48](https://github.com/awslabs/amazon-kinesis-client/issues/48) -* Upgraded the AWS Java SDK to version 1.11.271 - * [PR #287](https://github.com/awslabs/amazon-kinesis-client/pull/287) - -## Release 1.8.9 -* Allow disabling check for the case where a child shard has an open parent shard. - There is a race condition where it's possible for the a parent shard to appear open, while having child shards. This check can now be disabled by setting [`ignoreUnexpectedChildShards`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1037) to true. - * [PR #240](https://github.com/awslabs/amazon-kinesis-client/pull/240) - * [Issue #210](https://github.com/awslabs/amazon-kinesis-client/issues/210) -* Upgraded the AWS SDK for Java to 1.11.261 - * [PR #281](https://github.com/awslabs/amazon-kinesis-client/pull/281) - -## Release 1.8.8 -* Fixed issues with leases losses due to `ExpiredIteratorException` in `PrefetchGetRecordsCache` and `AsynchronousFetchingStrategy`. - PrefetchGetRecordsCache will request for a new iterator and start fetching data again. - * [PR#263](https://github.com/awslabs/amazon-kinesis-client/pull/263) -* Added warning message for long running tasks. - Logging long running tasks can be enabled by setting the following configuration property: - - | Name | Default | Description | - | ---- | ------- | ----------- | - | [`logWarningForTaskAfterMillis`](https://github.com/awslabs/amazon-kinesis-client/blob/3de901ea9327370ed732af86c4d4999c8d99541c/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1367) | Not set | Milliseconds after which the logger will log a warning message for the long running task | - - * [PR#259](https://github.com/awslabs/amazon-kinesis-client/pull/259) -* Handling spurious lease renewal failures gracefully. - Added better handling of DynamoDB failures when updating leases. These failures would occur when a request to DynamoDB appeared to fail, but was actually successful. - * [PR#247](https://github.com/awslabs/amazon-kinesis-client/pull/247) -* ShutdownTask gets retried if the previous attempt on the ShutdownTask fails. - * [PR#267](https://github.com/awslabs/amazon-kinesis-client/pull/267) -* Fix for using maxRecords from `KinesisClientLibConfiguration` in `GetRecordsCache` for fetching records. - * [PR#264](https://github.com/awslabs/amazon-kinesis-client/pull/264) - -## Release 1.8.7 -* Don't add a delay for synchronous requests to Kinesis - Removes a delay that had been added for synchronous `GetRecords` calls to Kinesis. - * [PR #256](https://github.com/awslabs/amazon-kinesis-client/pull/256) - -## Release 1.8.6 -* Add prefetching of records from Kinesis - Prefetching will retrieve and queue additional records from Kinesis while the application is processing existing records. - Prefetching can be enabled by setting [`dataFetchingStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1317) to `PREFETCH_CACHED`. Once enabled an additional fetching thread will be started to retrieve records from Kinesis. Retrieved records will be held in a queue until the application is ready to process them. - Pre-fetching supports the following configuration values: - - | Name | Default | Description | - | ---- | ------- | ----------- | - | [`dataFetchingStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1317) | `DEFAULT` | Which data fetching strategy to use | - | [`maxPendingProcessRecordsInput`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1296) | 3 | The maximum number of process records input that can be queued | - | [`maxCacheByteSize`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1307) | 8 MiB | The maximum number of bytes that can be queued | - | [`maxRecordsCount`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1326) | 30,000 | The maximum number of records that can be queued | - | [`idleMillisBetweenCalls`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1353) | 1,500 ms | The amount of time to wait between calls to Kinesis | - - * [PR #246](https://github.com/awslabs/amazon-kinesis-client/pull/246) - -## Release 1.8.5 (September 26, 2017) -* Only advance the shard iterator for the accepted response. - This fixes a race condition in the `KinesisDataFetcher` when it's being used to make asynchronous requests. The shard iterator is now only advanced when the retriever calls `DataFetcherResult#accept()`. - * [PR #230](https://github.com/awslabs/amazon-kinesis-client/pull/230) - * [Issue #231](https://github.com/awslabs/amazon-kinesis-client/issues/231) - -## Release 1.8.4 (September 22, 2017) -* Create a new completion service for each request. - This ensures that canceled tasks are discarded. This will prevent a cancellation exception causing issues processing records. - * [PR #227](https://github.com/awslabs/amazon-kinesis-client/pull/227) - * [Issue #226](https://github.com/awslabs/amazon-kinesis-client/issues/226) - -## Release 1.8.3 (September 22, 2017) -* Call shutdown on the retriever when the record processor is being shutdown - This fixes a bug that could leak threads if using the [`AsynchronousGetRecordsRetrievalStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/9a82b6bd05b3c9c5f8581af007141fa6d5f0fc4e/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java#L42) is being used. - The asynchronous retriever is only used when [`KinesisClientLibConfiguration#retryGetRecordsInSeconds`](https://github.com/awslabs/amazon-kinesis-client/blob/01d2688bc6e68fd3fe5cb698cb65299d67ac930d/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L227), and [`KinesisClientLibConfiguration#maxGetRecordsThreadPool`](https://github.com/awslabs/amazon-kinesis-client/blob/01d2688bc6e68fd3fe5cb698cb65299d67ac930d/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L230) are set. - * [PR #222](https://github.com/awslabs/amazon-kinesis-client/pull/222) - -## Release 1.8.2 (September 20, 2017) -* Add support for two phase checkpoints - Applications can now set a pending checkpoint, before completing the checkpoint operation. Once the application has completed its checkpoint steps, the final checkpoint will clear the pending checkpoint. - Should the checkpoint fail the attempted sequence number is provided in the [`InitializationInput#getPendingCheckpointSequenceNumber`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java#L81) otherwise the value will be null. - * [PR #188](https://github.com/awslabs/amazon-kinesis-client/pull/188) -* Support timeouts, and retry for GetRecords calls. - Applications can now set timeouts for GetRecord calls to Kinesis. As part of setting the timeout, the application must also provide a thread pool size for concurrent requests. - * [PR #214](https://github.com/awslabs/amazon-kinesis-client/pull/214) -* Notification when the lease table is throttled - When writes, or reads, to the lease table are throttled a warning will be emitted. If you're seeing this warning you should increase the IOPs for your lease table to prevent processing delays. - * [PR #212](https://github.com/awslabs/amazon-kinesis-client/pull/212) -* Support configuring the graceful shutdown timeout for MultiLang Clients - This adds support for setting the timeout that the Java process will wait for the MutliLang client to complete graceful shutdown. The timeout can be configured by adding `shutdownGraceMillis` to the properties file set to the number of milliseconds to wait. - * [PR #204](https://github.com/awslabs/amazon-kinesis-client/pull/204) - -## Release 1.8.1 (August 2, 2017) -* Support timeouts for calls to the MultiLang Daemon - This adds support for setting a timeout when dispatching records to the client record processor. If the record processor doesn't respond within the timeout the parent Java process will be terminated. This is a temporary fix to handle cases where the KCL becomes blocked while waiting for a client record processor. - The timeout for the this can be set by adding `timeoutInSeconds = `. The default for this is no timeout. - __Setting this can cause the KCL to exit suddenly, before using this ensure that you have an automated restart for your application__ - * [PR #195](https://github.com/awslabs/amazon-kinesis-client/pull/195) - * [Issue #185](https://github.com/awslabs/amazon-kinesis-client/issues/185) - -## Release 1.8.0 (July 25, 2017) -* Execute graceful shutdown on its own thread - * [PR #191](https://github.com/awslabs/amazon-kinesis-client/pull/191) - * [Issue #167](https://github.com/awslabs/amazon-kinesis-client/issues/167) -* Added support for controlling the size of the lease renewer thread pool - * [PR #177](https://github.com/awslabs/amazon-kinesis-client/pull/177) - * [Issue #171](https://github.com/awslabs/amazon-kinesis-client/issues/171) -* Require Java 8 and later - __Java 8 is now required for versions 1.8.0 of the amazon-kinesis-client and later.__ - * [PR #176](https://github.com/awslabs/amazon-kinesis-client/issues/176) - -## Release 1.7.6 (June 21, 2017) -* Added support for graceful shutdown in MultiLang Clients - * [PR #174](https://github.com/awslabs/amazon-kinesis-client/pull/174) - * [PR #182](https://github.com/awslabs/amazon-kinesis-client/pull/182) -* Updated documentation for `v2.IRecordProcessor#shutdown`, and `KinesisClientLibConfiguration#idleTimeBetweenReadsMillis` - * [PR #170](https://github.com/awslabs/amazon-kinesis-client/pull/170) -* Updated to version 1.11.151 of the AWS Java SDK - * [PR #183](https://github.com/awslabs/amazon-kinesis-client/pull/183) - -## Release 1.7.5 (April 7, 2017) -* Correctly handle throttling for DescribeStream, and save accumulated progress from individual calls. - * [PR #152](https://github.com/awslabs/amazon-kinesis-client/pull/152) -* Upgrade to version 1.11.115 of the AWS Java SDK - * [PR #155](https://github.com/awslabs/amazon-kinesis-client/pull/155) - -## Release 1.7.4 (February 27, 2017) -* Fixed an issue building JavaDoc for Java 8. - * [Issue #18](https://github.com/awslabs/amazon-kinesis-client/issues/18) - * [PR #141](https://github.com/awslabs/amazon-kinesis-client/pull/141) -* Reduce Throttling Messages to WARN, unless throttling occurs 6 times consecutively. - * [Issue #4](https://github.com/awslabs/amazon-kinesis-client/issues/4) - * [PR #140](https://github.com/awslabs/amazon-kinesis-client/pull/140) -* Fixed two bugs occurring in requestShutdown. - * Fixed a bug that prevented the worker from shutting down, via requestShutdown, when no leases were held. - * [Issue #128](https://github.com/awslabs/amazon-kinesis-client/issues/128) - * Fixed a bug that could trigger a NullPointerException if leases changed during requestShutdown. - * [Issue #129](https://github.com/awslabs/amazon-kinesis-client/issues/129) - * [PR #139](https://github.com/awslabs/amazon-kinesis-client/pull/139) -* Upgraded the AWS SDK Version to 1.11.91 - * [PR #138](https://github.com/awslabs/amazon-kinesis-client/pull/138) -* Use an executor returned from `ExecutorService.newFixedThreadPool` instead of constructing it by hand. - * [PR #135](https://github.com/awslabs/amazon-kinesis-client/pull/135) -* Correctly initialize DynamoDB client, when endpoint is explicitly set. - * [PR #142](https://github.com/awslabs/amazon-kinesis-client/pull/142) - -## Release 1.7.3 (January 9, 2017) -* Upgrade to the newest AWS Java SDK. - * [Amazon Kinesis Client Issue #27](https://github.com/awslabs/amazon-kinesis-client-python/issues/27) - * [PR #126](https://github.com/awslabs/amazon-kinesis-client/pull/126) - * [PR #125](https://github.com/awslabs/amazon-kinesis-client/pull/125) -* Added a direct dependency on commons-logging. - * [Issue #123](https://github.com/awslabs/amazon-kinesis-client/issues/123) - * [PR #124](https://github.com/awslabs/amazon-kinesis-client/pull/124) -* Make ShardInfo public to allow for custom ShardPrioritization strategies. - * [Issue #120](https://github.com/awslabs/amazon-kinesis-client/issues/120) - * [PR #127](https://github.com/awslabs/amazon-kinesis-client/pull/127) - -## Release 1.7.2 (November 7, 2016) -* MultiLangDaemon Feature Updates - The MultiLangDaemon has been upgraded to use the v2 interfaces, which allows access to enhanced checkpointing, and more information during record processor initialization. The MultiLangDaemon clients must be updated before they can take advantage of these new features. - -## Release 1.7.1 (November 3, 2016) -* General - * Allow disabling shard synchronization at startup. - * Applications can disable shard synchronization at startup. Disabling shard synchronization can application startup times for very large streams. - * [PR #102](https://github.com/awslabs/amazon-kinesis-client/pull/102) - * Applications can now request a graceful shutdown, and record processors that implement the IShutdownNotificationAware will be given a chance to checkpoint before being shutdown. - * This adds a [new interface](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/v2/IShutdownNotificationAware.java), and a [new method on Worker](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java#L539). - * [PR #109](https://github.com/awslabs/amazon-kinesis-client/pull/109) - * Solves [Issue #79](https://github.com/awslabs/amazon-kinesis-client/issues/79) -* MultiLangDaemon - * Applications can now use credential provides that accept string parameters. - * [PR #99](https://github.com/awslabs/amazon-kinesis-client/pull/99) - * Applications can now use different credentials for each service. - * [PR #111](https://github.com/awslabs/amazon-kinesis-client/pull/111) - -## Release 1.7.0 (August 22, 2016) -* Add support for time based iterators ([See GetShardIterator Documentation](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetShardIterator.html)) - * [PR #94](https://github.com/awslabs/amazon-kinesis-client/pull/94) - The `KinesisClientLibConfiguration` now supports providing an initial time stamp position. - * This position is only used if there is no current checkpoint for the shard. - * This setting cannot be used with DynamoDB Streams - Resolves [Issue #88](https://github.com/awslabs/amazon-kinesis-client/issues/88) -* Allow Prioritization of Parent Shards for Task Assignment - * [PR #95](https://github.com/awslabs/amazon-kinesis-client/pull/95) - The `KinesisClientLibconfiguration` now supports providing a `ShardPrioritization` strategy. This strategy controls how the `Worker` determines which `ShardConsumer` to call next. This can improve processing for streams that split often, such as DynamoDB Streams. -* Remove direct dependency on `aws-java-sdk-core`, to allow independent versioning. - * [PR #92](https://github.com/awslabs/amazon-kinesis-client/pull/92) - **You may need to add a direct dependency on aws-java-sdk-core if other dependencies include an older version.** - -## Release 1.6.5 (July 25, 2016) -* Change LeaseManager to call DescribeTable before attempting to create the lease table. - * [Issue #36](https://github.com/awslabs/amazon-kinesis-client/issues/36) - * [PR #41](https://github.com/awslabs/amazon-kinesis-client/pull/41) - * [PR #67](https://github.com/awslabs/amazon-kinesis-client/pull/67) -* Allow DynamoDB lease table name to be specified - * [PR #61](https://github.com/awslabs/amazon-kinesis-client/pull/61) -* Add approximateArrivalTimestamp for JsonFriendlyRecord - * [PR #86](https://github.com/awslabs/amazon-kinesis-client/pull/86) -* Shutdown lease renewal thread pool on exit. - * [PR #84](https://github.com/awslabs/amazon-kinesis-client/pull/84) -* Wait for CloudWatch publishing thread to finish before exiting. - * [PR #82](https://github.com/awslabs/amazon-kinesis-client/pull/82) -* Added unit, and integration tests for the library. - -## Release 1.6.4 (July 6, 2016) -* Upgrade to AWS SDK for Java 1.11.14 - * [Issue #74](https://github.com/awslabs/amazon-kinesis-client/issues/74) - * [Issue #73](https://github.com/awslabs/amazon-kinesis-client/issues/73) -* **Maven Artifact Signing Change** - * Artifacts are now signed by the identity `Amazon Kinesis Tools ` - -## Release 1.6.3 (May 12, 2016) -* Fix format exception caused by DEBUG log in LeaseTaker [Issue # 68](https://github.com/awslabs/amazon-kinesis-client/issues/68) - -## Release 1.6.2 (March 23, 2016) -* Support for specifying max leases per worker and max leases to steal at a time. -* Support for specifying initial DynamoDB table read and write capacity. -* Support for parallel lease renewal. -* Support for graceful worker shutdown. -* Change DefaultCWMetricsPublisher log level to debug. [PR # 49](https://github.com/awslabs/amazon-kinesis-client/pull/49) -* Avoid NPE in MLD record processor shutdown if record processor was not initialized. [Issue # 29](https://github.com/awslabs/amazon-kinesis-client/issues/29) - -## Release 1.6.1 (September 23, 2015) -* Expose [approximateArrivalTimestamp](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html) for Records in processRecords API call. - -## Release 1.6.0 (July 31, 2015) -* Restores compatibility with [dynamodb-streams-kinesis-adapter](https://github.com/awslabs/dynamodb-streams-kinesis-adapter) (which was broken in 1.4.0). - -## Release 1.5.1 (July 20, 2015) -* KCL maven artifact 1.5.0 does not work with JDK 7. This release addresses this issue. - -## Release 1.5.0 (July 9, 2015) -* **[Metrics Enhancements][kinesis-guide-monitoring-with-kcl]** - * Support metrics level and dimension configurations to control CloudWatch metrics emitted by the KCL. - * Add new metrics that track time spent in record processor methods. - * Disable WorkerIdentifier dimension by default. -* **Exception Reporting** — Do not silently ignore exceptions in ShardConsumer. -* **AWS SDK Component Dependencies** — Depend only on AWS SDK components that are used. - -## Release 1.4.0 (June 2, 2015) -* Integration with the **[Kinesis Producer Library (KPL)][kinesis-guide-kpl]** - * Automatically de-aggregate records put into the Kinesis stream using the KPL. - * Support checkpointing at the individual user record level when multiple user records are aggregated into one Kinesis record using the KPL. - - See [Consumer De-aggregation with the KCL][kinesis-guide-consumer-deaggregation] for details. - -## Release 1.3.0 (May 22, 2015) -* A new metric called "MillisBehindLatest", which tracks how far consumers are from real time, is now uploaded to CloudWatch. - -## Release 1.2.1 (January 26, 2015) -* **MultiLangDaemon** — Changes to the MultiLangDaemon to make it easier to provide a custom worker. - -## Release 1.2 (October 21, 2014) -* **Multi-Language Support** — Amazon KCL now supports implementing record processors in any language by communicating with the daemon over [STDIN and STDOUT][multi-lang-protocol]. Python developers can directly use the [Amazon Kinesis Client Library for Python][kclpy] to write their data processing applications. - -## Release 1.1 (June 30, 2014) -* **Checkpointing at a specific sequence number** — The IRecordProcessorCheckpointer interface now supports checkpointing at a sequence number specified by the record processor. -* **Set region** — KinesisClientLibConfiguration now supports setting the region name to indicate the location of the Amazon Kinesis service. The Amazon DynamoDB table and Amazon CloudWatch metrics associated with your application will also use this region setting. - [kinesis]: http://aws.amazon.com/kinesis [kinesis-forum]: http://developer.amazonwebservices.com/connect/forum.jspa?forumID=169 [kinesis-client-library-issues]: https://github.com/awslabs/amazon-kinesis-client/issues diff --git a/README.md b/README.md index 160ab682..90853c1f 100644 --- a/README.md +++ b/README.md @@ -66,166 +66,10 @@ The recommended way to use the KCL for Java is to consume it from Maven. ## Release Notes -### Release 2.4.5 (January 04, 2023) -* [#1014](https://github.com/awslabs/amazon-kinesis-client/pull/1014) Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request - -### Release 2.4.4 (December 23, 2022) -* [#1017](https://github.com/awslabs/amazon-kinesis-client/pull/1017) Upgrade aws sdk - * aws-java-sdk.version from 1.12.296 -> 1.12.370 - * awssdk.version from 2.17.268 -> 2.19.2 -* [#1020](https://github.com/awslabs/amazon-kinesis-client/pull/1020) Correct the KCL version in the main pom - -### Release 2.4.3 (September 6, 2022) -* [#980](https://github.com/awslabs/amazon-kinesis-client/pull/980) logback-classic: 1.2.9 -> 1.4.0 -* [#983](https://github.com/awslabs/amazon-kinesis-client/pull/983) - * protobuf-java: 3.19.2 -> 3.21.5 - * slf4j.version: 1.7.32 -> 2.0.0 - * schema-registry-serde: 1.1.9 -> 1.1.13 -* [#984](https://github.com/awslabs/amazon-kinesis-client/pull/984) awssdk.version from 2.17.108 to 2.17.267 -* [#987](https://github.com/awslabs/amazon-kinesis-client/pull/987) guava: 31.0.1-jre -> 31.1-jre -* [#988](https://github.com/awslabs/amazon-kinesis-client/pull/988) jcommander: 1.81 to 1.82 -* [#990](https://github.com/awslabs/amazon-kinesis-client/pull/990) Upgrade dependencies - * aws-java-sdk.version: 1.12.130 -> 1.12.296 - * lombok: 1.18.22 -> 1.18.24 - * rxjava: 3.1.3 -> 3.1.5 - * maven-resources-plugin: 2.6 -> 3.3.0 - * logback-classic: 1.4.0 -> 1.3.0 - * awssdk.version: 2.17.267 -> 2.17.268 - -### Release 2.4.2 (August 10, 2022) -* [#972](https://github.com/awslabs/amazon-kinesis-client/pull/972) Upgrade Lombok to version 1.18.24 - -### Latest Release 2.4.1 (March 24, 2022) -[Milestone#68](https://github.com/awslabs/amazon-kinesis-client/milestone/68) -* [#916](https://github.com/awslabs/amazon-kinesis-client/pull/916) Upgrade to rxjava3 - -### Release 2.4.0 (March 2, 2022) -[Milestone#67](https://github.com/awslabs/amazon-kinesis-client/milestone/67) -* [#894](https://github.com/awslabs/amazon-kinesis-client/pull/894) Bump protobuf-java from 3.19.1 to 3.19.2 -* [#924](https://github.com/awslabs/amazon-kinesis-client/pull/924) Support Protobuf Data format with Glue Schema Registry. - -### Latest Release 2.3.10 (January 4, 2022) -[Milestone#66](https://github.com/awslabs/amazon-kinesis-client/milestone/66) -* [#868](https://github.com/awslabs/amazon-kinesis-client/pull/868) Adding a new metric: Application-level MillisBehindLatest -* [#879](https://github.com/awslabs/amazon-kinesis-client/pull/879) Keep dependencies up-to-date -* [#886](https://github.com/awslabs/amazon-kinesis-client/pull/886) Get latest counter before attempting a take to ensure take succeeds -* [#888](https://github.com/awslabs/amazon-kinesis-client/pull/888) Configure dependabot for v1.x branch - -### Latest Release 2.3.9 (November 22, 2021) -[Milestone#65](https://github.com/awslabs/amazon-kinesis-client/milestone/65) -* [#866](https://github.com/awslabs/amazon-kinesis-client/pull/866) Update logback dependency. - -### Release 2.3.8 (October 27, 2021) -[Milestone#64](https://github.com/awslabs/amazon-kinesis-client/milestone/64) -* [#860](https://github.com/awslabs/amazon-kinesis-client/pull/860) Upgrade Glue schema registry from 1.1.4 to 1.1.5. -* [#861](https://github.com/awslabs/amazon-kinesis-client/pull/861) Revert [PR#847](https://github.com/awslabs/amazon-kinesis-client/pull/847) and added new tests. - -### Release 2.3.7 (October 11, 2021) -[Milestone#63](https://github.com/awslabs/amazon-kinesis-client/milestone/63) -* [#842](https://github.com/awslabs/amazon-kinesis-client/pull/842) Fixing typo is debug logs. -* [#846](https://github.com/awslabs/amazon-kinesis-client/pull/846) Fix DynamoDBLeaseTaker logging of available leases -* [#847](https://github.com/awslabs/amazon-kinesis-client/pull/847) Make use of Java 8 to simplify computeLeaseCounts() -* [#853](https://github.com/awslabs/amazon-kinesis-client/pull/853) Add configurable initial position for orphaned stream -* [#854](https://github.com/awslabs/amazon-kinesis-client/pull/854) Create DynamoDB tables on On-Demand billing mode by default. -* [#855](https://github.com/awslabs/amazon-kinesis-client/pull/855) Emit Glue Schema Registry usage metrics -* [#857](https://github.com/awslabs/amazon-kinesis-client/pull/857) Fix to shutdown PrefetchRecordsPublisher in gracefull manner -* [#858](https://github.com/awslabs/amazon-kinesis-client/pull/858) Upgrade AWS SDK version to 2.17.52. - -### Release 2.3.6 (July 9, 2021) -[Milestone#62](https://github.com/awslabs/amazon-kinesis-client/milestone/62) -* [#836](https://github.com/awslabs/amazon-kinesis-client/pull/836) Upgraded AWS SDK version to 2.16.98 -* [#835](https://github.com/awslabs/amazon-kinesis-client/pull/835) Upgraded Glue Schema Registry version to 1.1.1 -* [#828](https://github.com/awslabs/amazon-kinesis-client/pull/828) Modified wildcard imports to individual imports. -* [#817](https://github.com/awslabs/amazon-kinesis-client/pull/817) Updated the Worker shutdown logic to make sure that the `LeaseCleanupManager` also terminates all the threads that it has started. -* [#794](https://github.com/awslabs/amazon-kinesis-client/pull/794) Silence warning when there are no stale streams to delete. - -### Release 2.3.5 (June 14, 2021) -[Milestone#59](https://github.com/awslabs/amazon-kinesis-client/milestone/59) -* [#824](https://github.com/awslabs/amazon-kinesis-client/pull/824) Upgraded dependencies - * logback-classic version to 1.2.3 - * AWS Java SDK version to 1.12.3 - * AWS SDK version to 2.16.81 -* [#815](https://github.com/awslabs/amazon-kinesis-client/pull/815) Converted Future to CompletableFuture which helps in proper conversion to Scala using Scala Future Converters. -* [#810](https://github.com/awslabs/amazon-kinesis-client/pull/810) Bump commons-io from 2.6 to 2.7 -* [#804](https://github.com/awslabs/amazon-kinesis-client/pull/804) Allowing user to specify an initial timestamp in which daemon will process records. -* [#802](https://github.com/awslabs/amazon-kinesis-client/pull/802) Upgraded guava from 26.0-jre to 29.0-jre -* [#801](https://github.com/awslabs/amazon-kinesis-client/pull/801) Fixing a bug that causes to block indefinitely when trying to unlock a lock that isn't locked. -* [#762](https://github.com/awslabs/amazon-kinesis-client/pull/762) Added support for web identity token in multilang - -### Release 2.3.4 (February 19, 2021) -[Milestone#56](https://github.com/awslabs/amazon-kinesis-client/milestone/56) -* [#788](https://github.com/awslabs/amazon-kinesis-client/pull/788) Fixing a bug that caused paginated `ListShards` calls with the `ShardFilter` parameter to fail when the lease table was being initialized. - -### Release 2.3.3 (December 23, 2020) -[Milestone#55](https://github.com/awslabs/amazon-kinesis-client/milestone/55) -* Fixing bug in PrefetchRecordsPublisher which was causing retry storms if initial request fails. -* Fixing bug where idleTimeBetweenReadsInMillis property was ignored in PollingConfig. - -### Release 2.3.2 (November 19, 2020) -[Milestone#54](https://github.com/awslabs/amazon-kinesis-client/milestone/54) -* Adding support for Glue Schema Registry. Deserialize and read schemas associated with the records. -* Updating AWS SDK version to 2.15.31 - -### Release 2.3.1 (October 20, 2020) -[Milestone#53](https://github.com/awslabs/amazon-kinesis-client/milestone/53) -* Introducing support for processing multiple kinesis data streams with the same KCL 2.x for java consumer application - * To build a consumer application that can process multiple streams at the same time, you must implement a new - interface called MultistreamTracker (https://github.com/awslabs/amazon-kinesis-client/blob/0c5042dadf794fe988438436252a5a8fe70b6b0b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java) - - * MultistreamTracker will also publish various metrics around the current active streams being processed, the number - of streams which are deleted at this time period or are pending deletion. - -### Release 2.3.0 (August 17, 2020) -* [Milestone#52](https://github.com/awslabs/amazon-kinesis-client/milestones/52) - -* Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. -* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. - * Currently, the following shard filters are supported: - * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. - * `AT_LATEST` - the response includes only the currently open shards of the data stream. - * `AT_TIMESTAMP` - the response includes all shards whose start timestamp is less than or equal to the given timestamp and end timestamp is greater than or equal to the given timestamp or still open. - * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `RetrievalConfig#initialPositionInStreamExtended`. - * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). - -* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. - * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. - * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). - -* KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. `PeriodicShardSyncManager` is the new component that is responsible for running periodic lease/shard scans. - * New configuration options are available to configure `PeriodicShardSyncManager` in `LeaseManagementConfig` - - | Name | Default | Description | - | ----------------------------------------------------- | ----------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | - | leasesRecoveryAuditorExecutionFrequencyMillis | 120000 (2 minutes) | Frequency (in millis) of the auditor job to scan for partial leases in the lease table. If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on leasesRecoveryAuditorInconsistencyConfidenceThreshold. | - | leasesRecoveryAuditorInconsistencyConfidenceThreshold | 3 | Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, then it would trigger a shard sync | - - * New CloudWatch metrics are also now emitted to monitor the health of `PeriodicShardSyncManager`: - - | Name | Description | - | --------------------------- | ------------------------------------------------------ | - | NumStreamsWithPartialLeases | Number of streams that had holes in their hash ranges. | - | NumStreamsToSync | Number of streams which underwent a full shard sync. | - -* Introducing deferred lease cleanup. Leases will be deleted asynchronously by `LeaseCleanupManager` upon reaching `SHARD_END`, when a shard has either expired past the stream’s retention period or been closed as the result of a resharding operation. - * New configuration options are available to configure `LeaseCleanupManager`. - - | Name | Default | Description | - | ----------------------------------- | ---------- | --------------------------------------------------------------------------------------------------------- | - | leaseCleanupIntervalMillis | 1 minute | Interval at which to run lease cleanup thread. | - | completedLeaseCleanupIntervalMillis | 5 minutes | Interval at which to check if a lease is completed or not. | - | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | - -* Introducing _experimental_ support for multistreaming, allowing a single KCL application to multiplex processing multiple streams. - * New configuration options are available to enable multistreaming in `RetrievalConfig#appStreamTracker`. - -* Fixing a bug in `PrefetchRecordsPublisher` restarting while it was already running. -* Including an optimization to `HierarchicalShardSyncer` to only create leases for one layer of shards. -* Adding support to prepare and commit lease checkpoints with arbitrary bytes. - * This allows checkpointing of an arbitrary byte buffer up to the maximum permitted DynamoDB item size ([currently 400 KB as of release](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Limits.html)), and can be used for recovery by passing a serialized byte buffer to `RecordProcessorCheckpointer#prepareCheckpoint` and `RecordProcessorCheckpointer#checkpoint`. -* Upgrading version of AWS SDK to 2.14.0. -* [#725](https://github.com/awslabs/amazon-kinesis-client/pull/725) Allowing KCL to consider lease tables in `UPDATING` healthy. - -### For remaining release notes check **[CHANGELOG.md][changelog-md]**. +| KCL Version | Changelog | +| --- | --- | +| 2.x | [master/CHANGELOG.md](CHANGELOG.md) | +| 1.x | [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md) | [kinesis]: http://aws.amazon.com/kinesis [kinesis-forum]: http://developer.amazonwebservices.com/connect/forum.jspa?forumID=169 @@ -240,5 +84,4 @@ The recommended way to use the KCL for Java is to consume it from Maven. [kinesis-guide-consumer-deaggregation]: http://docs.aws.amazon.com//kinesis/latest/dev/kinesis-kpl-consumer-deaggregation.html [kclpy]: https://github.com/awslabs/amazon-kinesis-client-python [multi-lang-protocol]: https://github.com/awslabs/amazon-kinesis-client/blob/master/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/package-info.java -[changelog-md]: https://github.com/awslabs/amazon-kinesis-client/blob/master/CHANGELOG.md [migration-guide]: https://docs.aws.amazon.com/streams/latest/dev/kcl-migration.html From 1c8bd8e71efabc5ba4577f09c6a16c5cb6e0e033 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 13 Feb 2023 13:58:02 -0500 Subject: [PATCH 22/88] Minor optimizations (e.g., calculate-once, `put` instead of `get+put`) (#1041) and code cleanup (e.g., removed unused imports, updated Javadoc). No functional change. --- .../coordinator/PeriodicShardSyncManager.java | 10 ++--- .../leases/HierarchicalShardSyncer.java | 6 +-- .../software/amazon/kinesis/leases/Lease.java | 30 ++++++--------- .../kinesis/leases/LeaseManagementConfig.java | 8 ---- .../amazon/kinesis/leases/LeaseRefresher.java | 2 - .../leases/dynamodb/DynamoDBLeaseTaker.java | 14 ++----- .../kinesis/lifecycle/InitializeTask.java | 1 - .../kinesis/lifecycle/ShardConsumer.java | 4 +- .../kinesis/retrieval/RecordsPublisher.java | 5 --- .../kinesis/retrieval/RetrievalConfig.java | 4 +- .../retrieval/polling/PollingConfig.java | 5 --- .../polling/PrefetchRecordsPublisher.java | 10 ++--- ...ynchronousGetRecordsRetrievalStrategy.java | 1 - .../dynamodb/DynamoDBLeaseRefresherTest.java | 1 - .../dynamodb/DynamoDBLeaseTakerTest.java | 37 +------------------ .../kinesis/lifecycle/ShutdownTaskTest.java | 4 -- 16 files changed, 30 insertions(+), 112 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java index a2d05e6d..a885c4d9 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java @@ -76,7 +76,7 @@ class PeriodicShardSyncManager { @VisibleForTesting static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); static final String PERIODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager"; - private Map hashRangeHoleTrackerMap = new HashMap<>(); + private final Map hashRangeHoleTrackerMap = new HashMap<>(); private final String workerId; private final LeaderDecider leaderDecider; @@ -142,15 +142,12 @@ class PeriodicShardSyncManager { /** * Runs shardSync once * Does not schedule periodic shardSync - * @return the result of the task */ public synchronized void syncShardsOnce() throws Exception { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - for(Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { - final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); - log.info("Syncing Kinesis shard info for " + streamIdentifier); - final StreamConfig streamConfig = streamConfigEntry.getValue(); + for (StreamConfig streamConfig : currentStreamConfigMap.values()) { + log.info("Syncing Kinesis shard info for {}", streamConfig); final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfig); final TaskResult taskResult = shardSyncTaskManager.callShardSyncTask(); if (taskResult.getException() != null) { @@ -283,7 +280,6 @@ class PeriodicShardSyncManager { "Detected same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + " times. Shard sync will be initiated when threshold reaches " + leasesRecoveryAuditorInconsistencyConfidenceThreshold); - } else { // If hole is not present, clear any previous tracking for this stream and return false; hashRangeHoleTrackerMap.remove(streamIdentifier); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index 068db578..e44125a5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -62,7 +62,7 @@ import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRan * Helper class to sync leases with shards of the Kinesis stream. * It will create new leases/activities when it discovers new Kinesis shards (bootstrap/resharding). * It deletes leases for shards that have been trimmed from Kinesis, or if we've completed processing it - * and begun processing it's child shards. + * and begun processing its child shards. */ @Slf4j @KinesisClientInternalApi @@ -432,7 +432,7 @@ public class HierarchicalShardSyncer { if (!shardIdsOfCurrentLeases.contains(parentShardId)) { Lease lease = shardIdToLeaseMapOfNewShards.get(parentShardId); - /** + /* * If the lease for the parent shard does not already exist, there are two cases in which we * would want to create it: * - If we have already marked the parentShardId for lease creation in a prior recursive @@ -454,7 +454,7 @@ public class HierarchicalShardSyncer { } } - /** + /* * 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 diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java index 4074db22..f761a9a7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java @@ -42,7 +42,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "childShardIds", "pendingCheckpointState", "isMarkedForLeaseSteal"}) @ToString public class Lease { - /* + /** * See javadoc for System.nanoTime - summary: * * Sometimes System.nanoTime's return values will wrap due to overflow. When they do, the difference between two @@ -51,62 +51,57 @@ public class Lease { private static final long MAX_ABS_AGE_NANOS = TimeUnit.DAYS.toNanos(365); /** - * @return leaseKey - identifies the unit of work associated with this lease. + * Identifies the unit of work associated with this lease. */ private String leaseKey; /** - * @return current owner of the lease, may be null. + * Current owner of the lease, may be null. */ private String leaseOwner; /** - * @return leaseCounter is incremented periodically by the holder of the lease. Used for optimistic locking. + * LeaseCounter is incremented periodically by the holder of the lease. Used for optimistic locking. */ private Long leaseCounter = 0L; - /* + /** * This field is used to prevent updates to leases that we have lost and re-acquired. It is deliberately not * persisted in DynamoDB and excluded from hashCode and equals. */ private UUID concurrencyToken; - /* + /** * This field is used by LeaseRenewer and LeaseTaker to track the last time a lease counter was incremented. It is * deliberately not persisted in DynamoDB and excluded from hashCode and equals. */ private Long lastCounterIncrementNanos; /** - * @return most recently application-supplied checkpoint value. During fail over, the new worker will pick up after + * Most recently application-supplied checkpoint value. During fail over, the new worker will pick up after * the old worker's last checkpoint. */ private ExtendedSequenceNumber checkpoint; /** - * @return pending checkpoint, possibly null. + * Pending checkpoint, possibly null. */ private ExtendedSequenceNumber pendingCheckpoint; /** - * Last pending application state. Deliberately excluded from hashCode and equals. - * - * @return pending checkpoint state, possibly null. + * Last pending checkpoint state, possibly null. Deliberately excluded from hashCode and equals. */ private byte[] pendingCheckpointState; - /** * Denotes whether the lease is marked for stealing. Deliberately excluded from hashCode and equals and * not persisted in DynamoDB. - * - * @return flag for denoting lease is marked for stealing. */ @Setter private boolean isMarkedForLeaseSteal; /** - * @return count of distinct lease holders between checkpoints. + * Count of distinct lease holders between checkpoints. */ private Long ownerSwitchesSinceCheckpoint = 0L; - private Set parentShardIds = new HashSet<>(); - private Set childShardIds = new HashSet<>(); + private final Set parentShardIds = new HashSet<>(); + private final Set childShardIds = new HashSet<>(); private HashKeyRangeForLease hashKeyRangeForLease; /** @@ -319,5 +314,4 @@ public class Lease { return new Lease(this); } - } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java index 89e6a3bf..4f2d3a2b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java @@ -64,22 +64,16 @@ public class LeaseManagementConfig { /** * Name of the table to use in DynamoDB - * - * @return String */ @NonNull private final String tableName; /** * Client to be used to access DynamoDB service. - * - * @return {@link DynamoDbAsyncClient} */ @NonNull private final DynamoDbAsyncClient dynamoDBClient; /** * Client to be used to access Kinesis Data Streams service. - * - * @return {@link KinesisAsyncClient} */ @NonNull private final KinesisAsyncClient kinesisClient; @@ -90,8 +84,6 @@ public class LeaseManagementConfig { private String streamName; /** * Used to distinguish different workers/processes of a KCL application. - * - * @return String */ @NonNull private final String workerIdentifier; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java index 2fca59c7..7ec5b5ec 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java @@ -210,8 +210,6 @@ public interface LeaseRefresher { * Update application-specific fields of the given lease in DynamoDB. Does not update fields managed by the leasing * library such as leaseCounter, leaseOwner, or leaseKey. * - * @return true if update succeeded, false otherwise - * * @throws InvalidStateException if lease table does not exist * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws DependencyException if DynamoDB update fails in an unexpected way diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java index 4a4f086f..9fb91f14 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java @@ -73,7 +73,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { private long veryOldLeaseDurationNanosMultiplier = 3; private long lastScanTimeNanos = 0L; - public DynamoDBLeaseTaker(LeaseRefresher leaseRefresher, String workerIdentifier, long leaseDurationMillis, final MetricsFactory metricsFactory) { this.leaseRefresher = leaseRefresher; @@ -184,7 +183,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { MetricsUtil.addSuccessAndLatency(scope, "ListLeases", success, startTime, MetricsLevel.DETAILED); } - if (lastException != null) { log.error("Worker {} could not scan leases table, aborting TAKE_LEASES_DIMENSION. Exception caught by" + " last retry:", workerIdentifier, lastException); @@ -319,8 +317,7 @@ public class DynamoDBLeaseTaker implements LeaseTaker { for (Lease lease : freshList) { String leaseKey = lease.leaseKey(); - Lease oldLease = allLeases.get(leaseKey); - allLeases.put(leaseKey, lease); + final Lease oldLease = allLeases.put(leaseKey, lease); notUpdated.remove(leaseKey); if (oldLease != null) { @@ -384,7 +381,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { Set leasesToTake = new HashSet<>(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION); MetricsUtil.addWorkerIdentifier(scope, workerIdentifier); - List veryOldLeases = new ArrayList<>(); final int numAvailableLeases = expiredLeases.size(); int numLeases = 0; @@ -402,7 +398,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { return leasesToTake; } - int target; if (numWorkers >= numLeases) { // If we have n leases and n or more workers, each worker can have up to 1 lease, including myself. @@ -435,9 +430,9 @@ public class DynamoDBLeaseTaker implements LeaseTaker { // If there are leases that have been expired for an extended period of // time, take them with priority, disregarding the target (computed // later) but obeying the maximum limit per worker. - veryOldLeases = allLeases.values().stream() - .filter(lease -> System.nanoTime() - lease.lastCounterIncrementNanos() - > veryOldLeaseDurationNanosMultiplier * leaseDurationNanos) + final long nanoThreshold = System.nanoTime() - (veryOldLeaseDurationNanosMultiplier * leaseDurationNanos); + final List veryOldLeases = allLeases.values().stream() + .filter(lease -> nanoThreshold > lease.lastCounterIncrementNanos()) .collect(Collectors.toList()); if (!veryOldLeases.isEmpty()) { @@ -481,7 +476,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { workerIdentifier, numLeases, numAvailableLeases, numWorkers, target, myCount, leasesToTake.size()); } - } finally { scope.addData("ExpiredLeases", expiredLeases.size(), StandardUnit.COUNT, MetricsLevel.SUMMARY); scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java index 4108dd9b..7816c1e1 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java @@ -21,7 +21,6 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; -import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.metrics.MetricsFactory; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java index b6e7c068..a575a953 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java @@ -16,7 +16,6 @@ package software.amazon.kinesis.lifecycle; import java.time.Duration; import java.time.Instant; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -33,7 +32,6 @@ import lombok.Getter; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; -import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; import software.amazon.kinesis.leases.ShardInfo; @@ -75,7 +73,7 @@ public class ShardConsumer { private volatile Instant taskDispatchedAt; private volatile boolean taskIsRunning = false; - /* + /** * Tracks current state. It is only updated via the consumeStream/shutdown APIs. Therefore we don't do * much coordination/synchronization to handle concurrent reads/updates. */ diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RecordsPublisher.java index 5fc029b4..98c0375e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RecordsPublisher.java @@ -21,15 +21,11 @@ import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.RequestDetails; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.Optional; - /** * Provides a record publisher that will retrieve records from Kinesis for processing */ public interface RecordsPublisher extends Publisher { - - /** * Initializes the publisher with where to start processing. If there is a stored sequence number the publisher will * begin from that sequence number, otherwise it will use the initial position. @@ -47,7 +43,6 @@ public interface RecordsPublisher extends Publisher { */ void restartFrom(RecordsRetrieved recordsRetrieved); - /** * Shutdowns the publisher. Once this method returns the publisher should no longer provide any records. */ diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 000b71b7..f45fa80d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -108,7 +108,7 @@ public class RetrievalConfig { * * @deprecated Initial stream position is now handled by {@link StreamTracker}. * @see StreamTracker#orphanedStreamInitialPositionInStream() - * @see StreamTracker#createConfig(StreamIdentifier) + * @see StreamTracker#createStreamConfig(StreamIdentifier) */ @Deprecated private InitialPositionInStreamExtended initialPositionInStreamExtended = InitialPositionInStreamExtended @@ -138,7 +138,7 @@ public class RetrievalConfig { * * @deprecated Initial stream position is now handled by {@link StreamTracker}. * @see StreamTracker#orphanedStreamInitialPositionInStream() - * @see StreamTracker#createConfig(StreamIdentifier) + * @see StreamTracker#createStreamConfig(StreamIdentifier) */ @Deprecated public RetrievalConfig initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java index 181cea76..a37e7121 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java @@ -18,7 +18,6 @@ package software.amazon.kinesis.retrieval.polling; import java.time.Duration; import java.util.Optional; import java.util.function.Function; -import lombok.Data; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NonNull; @@ -47,8 +46,6 @@ public class PollingConfig implements RetrievalSpecificConfig { Function dataFetcherProvider; /** * Name of the Kinesis stream. - * - * @return String */ private String streamName; @@ -63,8 +60,6 @@ public class PollingConfig implements RetrievalSpecificConfig { /** * Client used to access to Kinesis service. - * - * @return {@link KinesisAsyncClient} */ @NonNull private final KinesisAsyncClient kinesisClient; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index 1a49cdfb..ab406244 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -70,7 +70,7 @@ import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDelivery * i.e. the byte size of the records stored in the cache and maxRecordsCount i.e. the max number of records that should * be present in the cache across multiple GetRecordsResult object. If no data is available in the cache, the call from * the record processor is blocked till records are retrieved from Kinesis. - * + *

* There are three threads namely publisher, demand-notifier and ack-notifier which will contend to drain the events * to the Subscriber (ShardConsumer in KCL). */ @@ -81,9 +81,9 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { // Since this package is being used by all KCL clients keeping the upper threshold of 60 seconds private static final long DEFAULT_AWAIT_TERMINATION_TIMEOUT_MILLIS = 60_000L; - private int maxPendingProcessRecordsInput; - private int maxByteSize; - private int maxRecordsCount; + private final int maxPendingProcessRecordsInput; + private final int maxByteSize; + private final int maxRecordsCount; private final int maxRecordsPerCall; private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final ExecutorService executorService; @@ -447,7 +447,6 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { } - private class DefaultGetRecordsCacheDaemon implements Runnable { volatile boolean isShutdown = false; @@ -483,7 +482,6 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation); if (publisherSession.prefetchCounters().shouldGetNewRecords()) { try { - sleepBeforeNextCall(); GetRecordsResponse getRecordsResult = getRecordsRetrievalStrategy.getRecords(maxRecordsPerCall); lastSuccessfulCall = Instant.now(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java index 7f3b54d5..cdf03fac 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java @@ -14,7 +14,6 @@ */ package software.amazon.kinesis.retrieval.polling; -import java.util.Optional; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java index beed73f2..ac814d75 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java @@ -22,7 +22,6 @@ import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerTest.java index 193970f6..b6e74a6b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerTest.java @@ -15,10 +15,8 @@ package software.amazon.kinesis.leases.dynamodb; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -28,19 +26,14 @@ import java.util.concurrent.Callable; import java.util.function.Function; import java.util.stream.Collectors; -import junit.framework.Assert; - -import org.junit.After; -import org.junit.AfterClass; +import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; -import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseTaker; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.NullMetricsScope; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @@ -68,34 +61,6 @@ public class DynamoDBLeaseTakerTest { this.dynamoDBLeaseTaker = new DynamoDBLeaseTaker(leaseRefresher, WORKER_IDENTIFIER, LEASE_DURATION_MILLIS, metricsFactory); } - /** - * @throws java.lang.Exception - */ - @BeforeClass - public static void setUpBeforeClass() throws Exception { - } - - /** - * @throws java.lang.Exception - */ - @AfterClass - public static void tearDownAfterClass() throws Exception { - } - - /** - * @throws java.lang.Exception - */ - @Before - public void setUp() throws Exception { - } - - /** - * @throws java.lang.Exception - */ - @After - public void tearDown() throws Exception { - } - /** * Test method for {@link DynamoDBLeaseTaker#stringJoin(java.util.Collection, java.lang.String)}. */ diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java index 688bd199..6617984d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java @@ -17,10 +17,8 @@ package software.amazon.kinesis.lifecycle; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; @@ -48,7 +46,6 @@ import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; -import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCleanupManager; @@ -265,7 +262,6 @@ public class ShutdownTaskTest { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2")); - Lease parentLease = LeaseHelper.createLease("shardId-1", "leaseOwner", Collections.emptyList()); when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); when(leaseCoordinator.getCurrentlyHeldLease("shardId-1")) .thenReturn(null, null, null, null, null, null, null, null, null, null, null); From 70cfc7d7ce1bd782a49916394bce592b29f548b4 Mon Sep 17 00:00:00 2001 From: ZeyuLi-AWS <125080976+ZeyuLi-AWS@users.noreply.github.com> Date: Mon, 20 Feb 2023 15:53:57 -0800 Subject: [PATCH 23/88] Fixed duplication of project version in children pom.xml (#1045) --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- pom.xml | 3 ++- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 879f7cf9..8ec7c5a7 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.6-SNAPSHOT + ${revision} 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 19119ed0..4fe771b7 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.6-SNAPSHOT + ${revision} amazon-kinesis-client diff --git a/pom.xml b/pom.xml index 8475338f..d7b6d42a 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.6-SNAPSHOT + ${revision} The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. @@ -33,6 +33,7 @@ + 2.4.6-SNAPSHOT 2.19.31 From d8aa784f17920804e12deca3d6c982ce19da3079 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 27 Feb 2023 13:02:36 -0500 Subject: [PATCH 24/88] Removed a `.swp` file, and updated `.gitignore`. (#1043) --- .gitignore | 3 ++- .log.swp | Bin 32768 -> 0 bytes 2 files changed, 2 insertions(+), 1 deletion(-) delete mode 100644 .log.swp diff --git a/.gitignore b/.gitignore index 7fd632f3..c4d508a5 100644 --- a/.gitignore +++ b/.gitignore @@ -2,4 +2,5 @@ target/ AwsCredentials.properties .idea *.iml -.DS_Store \ No newline at end of file +*.swp +.DS_Store diff --git a/.log.swp b/.log.swp deleted file mode 100644 index bdb60bb3666b44a9fe4b8aecb4b884dace9bfec5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 32768 zcmeI)37FL683*v7c%ocVso)VVIcJ7lKm-jpJ zWfC&E=Y)}?&11twhP;{^#shEHMBB|a_L|$vFr3P0D(OUH9_{W`)Vpsy9_K7V<_2oW}1{xW-{0^3CK;$UlMq5Zi4*{ttL7 zcvtd|;fKQelm8vw8eU5N5nSV6MZO6hM*3OgACfO2{~LKN`CrMKq8;ID#s}~aw%?EZ zeYlq2CjSdOg7oK;zX$IQUqZeS-UI#?`3AU_?*Mc-E&sdZ$B?gwcR>Cz@^>ixZ1Qz* zt)B4F@_&MSHTfFyRpcAUSCW55{sMWE z{rrMze4mGF{Ej4FLFs#uKSw^C{8_k;-=~s4L!Kmmn)07X{uErxev ze;lsmf0g_(%6|j-qi}7%+sPlH{9Ei_zkYrL*Y??o{9*EA$(NIlBL6k{8RQR50Jk?en0to^84Ugzgx%`!w*LPKJbA0^>Z(IYw~;GO_9DM z`LEz|i7NjpCcm3}F!>^Q3#1=MeivNZPmKIdxXvG{$rr+pM*j22?|^rMKSX{zT*t>+ z^4s8DkbWEat#Cblw8K1I$FE!99g%)C`OWZ7@H5G8f_H{rO}+r$4gMJUjqtAU_sDO6 zYx$bvJVMKVJzV44oBTSsmhUw3YvCH7ndI}~y8ri*Ujx_q!y59d;aa{=$*-dAw>`)Y zgzn#!lzs&H6>u$IlKgT?e>wSOaBV-!$>+g)pnht}=fc~<4?;)R@?Q$q`A=u^OW<1m zlgWPxH<5o0`Ni=3#Vh&Fwd5DU+adi*@(bbGK0hG;1-v`bx57ZA`!|Qu46EG=bO zD*b+oj9r^#Ss=?8S-zBGiFZ7?RhF@`jF9CGIsV@yOIVgxvRokZwMMdR+ehkNmM*d^ zmO1`HSw_inm@JFroaJK)fcG1Nw?8#ndAscO_hCiZ$T7pmPBJ6o9V_cZ{>7~k4c{=~ zWMh$J)J{dJqSeuelP-6sMza>~u8837ib%qWI+hbT!M3u|Y^>6bMH8i!(QLf5CKa=? zVJ~foS(;8-rk%>9!n4A7qjmeE> zD=aq>OGF)ME?T!kJqogqq&5uU+QWeZ#LM-P03HQ)^}%$oiS7Clxa<~9oGv+*78F! z)RxjoE2;Tbp(yk#%~ry3%Aj#$MvfUV$sF%^8_Xo4uC%M9X{BOmX_^&gm+>bL4;^bp zV{R^*NYr#Sr9NWmOpRANu9cP06>{rg+F>hfPR`khxM@2(Gm}cUQgOK#s)1-0QfoMi z!^e~qR+@!?7FO+1NK5^e6xaBbe5(MWEmc@4Z?Lf9A-{H<{6H0Qt)v$|X;sm9+=|y} z+ZkKJRJ5}_r*p11BK>DerLAm;R9?QL6)Nt()(WI3UP=6sIX}$#jmldo&CCmES_b!; zoslX8)$5K3xO_Y0_e^xiqHxb}&k{c?ts=9F)>=pgBS!{$xjj#Yggt6aL#>3B@0Z`^ z=;k*ft&KXviX6~VJ3B$2l9;z*X9vjtxkts7ym$L5GpnLmbJA|V=tmQPxG*)tqyY=1v?wu#;)ShcfNV8z$A3vU|@9cy-OCEu=gZetJI z-EQ))TSdEkE;GG?edlejfhJdIe`Q4twY@!TdHMaV)B3d5^u6JRkd!HboscOy&_}*uSKCc?<{sp?yKD+pCN)68(BFaNT+(3MMx2H*o zEIHGVOeQC+q+?E&NxOIIA**ZTA1SIUh4ZGFGE_~r6SiAprpwJ5nXdRH`kKvl&QGqw z{(;WHdl);}Ie$u!&Q^q@nW&sjg!4y5{c?$aj4aHf(B-||NZO7!`wPotRYpzA?A5zh z51Hg8a>tm`M{xb$Zx_M!|N8wSxc=|=kKp>h z-!8td*Z=kYh;MrRU(f$%`qleYXMgvML3IDXdE}MkGvHm3elGdB@DO}C`8n_~d;|H} zaJ~QR5Znl+=Z9y(_5S$IUNm*BSDuaGhVxA+Lt({r5MJ=V<#& z$X&SJf3}L;fn&AtE_s&n-%NfcT=(xF+$g8<&A{bWjQpzuc^Zz@MmO>l<$oM`60YSR zN1lM|{mGNbr@(Prncs&<>b$hTX1bZuad{% z+I}{Y$H>1RpA6UeO-lq)>o*G5_;n)x8C>HRA^#~{`_~BaGvM03envhCuKPEY{B%lx z33(Y@+t2Ofr@^t>SW12>T+6?Ld?H-;|84S9DE(IQ32?bBD*tMZiqQCvhim)~CqJ3; zKbpK0j%qgsl8=LH`#OpICvY8KE%KA#8s8f7v2YCA#)af#;99@)$w$LA{)@>^glqYq zCm#jJYU54vA5;2|$VbApeKtlQw0=&2Ykb<0kD&Y`OkM)l`aPQbN0fhm@?+uJ{)dwHg=_pyCO-zQ^%o`Y1K0kaCGQQ#uwtA|-iy*- zN?r^{SdIDQMR2X(d&qmjvD$c+JVO3Dd6>4pi97_?`r8Wwh1PEmxQ-uf$-C3`i^#jt z_J@;qrSzwfcOg%ccZOrNF^l|YxSl^#6jbB$L%6p8!^vC2HGW0phmwyV{{i_V@>b*y`61*NlD8zkiM$2*GVm@+RcRksnC@6Y>MdE6DeUYyUcjd_TDM?`z2Sh3ojc zgnS>`{tEKOaBY8YlJ8CaF*%z3E^m7NzZaEnU+k#HzY$#PuMN2Y*Z-edO#aoE{ueZU zW5~aR>;6@e?|^Ik&LRH-j&5ySM*cbZUF6%zpCI2xzLxwm@(;E!<)cgQ~`zli+rBg#3@>E6M*rUQ51~d@K2DzeE0O@@?b~kvB&}()cbT?@YcFuJf}2$>L;eR+{?*8TGai8J z`QsJj_ro>*caYx)*Z4m|zLSCV%jzk+-q`Q_vj$S)&L zlFuWbMLw7Odh$!jmylmVzLNZxGZdyLdyKz%) zZqxIpMe_H}vW%6*vm<2*UO@zc2m}!bA`nC%h(Hj5AOb-If(Qf=2qF+f;D0m%Wy8je z8#`{2Jkl{)oih8$BeI3HvG6%L%%<$Tpsi%PX^3-kE7N{{OR)mdw7p8H06nW z-h+j6^4xXr`G5I`0s4Jf4x1LUGTuYnjei2h8D_Tt From cd80c93966a310cd6342f66d55c23ba469df98bd Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 27 Feb 2023 13:15:30 -0500 Subject: [PATCH 25/88] Exposed convenience method of `ExtendedSequenceNumber#isSentinelCheckpoint()` (#1053) + fixed unrelated parameterized log message in `ShardSyncTaskManager` --- .../kinesis/leases/ShardSyncTaskManager.java | 3 +- .../leases/exceptions/ShardSyncer.java | 3 +- .../amazon/kinesis/lifecycle/ProcessTask.java | 4 +- .../retrieval/kpl/ExtendedSequenceNumber.java | 89 +++++++------------ .../kpl/ExtendedSequenceNumberTest.java | 36 ++++++++ 5 files changed, 70 insertions(+), 65 deletions(-) create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumberTest.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java index e03046a0..1caf0629 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java @@ -205,7 +205,8 @@ public class ShardSyncTaskManager { private void handlePendingShardSyncs(Throwable exception, TaskResult taskResult) { if (exception != null || taskResult.getException() != null) { - log.error("Caught exception running {} task: ", currentTask.taskType(), exception != null ? exception : taskResult.getException()); + log.error("Caught exception running {} task: {}", currentTask.taskType(), + exception != null ? exception : taskResult.getException()); } // Acquire lock here. If shardSyncRequestPending is false in this completionStage and // submitShardSyncTask is invoked, before completion stage exits (future completes) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java index f7ec12c5..792555d2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java @@ -12,14 +12,13 @@ import software.amazon.kinesis.metrics.MetricsScope; * Helper class to sync leases with shards of the Kinesis stream. * It will create new leases/activities when it discovers new Kinesis shards (bootstrap/resharding). * It deletes leases for shards that have been trimmed from Kinesis, or if we've completed processing it - * and begun processing it's child shards. + * and begun processing its child shards. * *

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

*/ @Deprecated public class ShardSyncer { private static final HierarchicalShardSyncer HIERARCHICAL_SHARD_SYNCER = new HierarchicalShardSyncer(); - private static final boolean garbageCollectLeases = true; /** *

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

diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java index e4b38815..9f616b0d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java @@ -113,7 +113,7 @@ public class ProcessTask implements ConsumerTask { */ @Override public TaskResult call() { - /** + /* * NOTE: the difference between appScope and shardScope is, appScope doesn't have shardId as a dimension, * therefore all data added to appScope, although from different shard consumer, will be sent to the same metric, * which is the app-level MillsBehindLatest metric. @@ -180,8 +180,6 @@ public class ProcessTask implements ConsumerTask { } } - - private List deaggregateAnyKplRecords(List records) { if (shard == null) { return aggregatorUtil.deaggregate(records); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java index 0c1c4a28..fed58739 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java @@ -15,8 +15,12 @@ package software.amazon.kinesis.retrieval.kpl; import java.math.BigInteger; +import java.util.Arrays; +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; -//import com.amazonaws.services.kinesis.clientlibrary.lib.worker.String; +import lombok.EqualsAndHashCode; import software.amazon.kinesis.checkpoint.SentinelCheckpoint; /** @@ -28,10 +32,8 @@ import software.amazon.kinesis.checkpoint.SentinelCheckpoint; * user record therefore has an integer sub-sequence number, in addition to the * regular sequence number of the Kinesis record. The sub-sequence number is * used to checkpoint within an aggregated record. - * - * @author daphnliu - * */ +@EqualsAndHashCode public class ExtendedSequenceNumber implements Comparable { private final String sequenceNumber; private final long subSequenceNumber; @@ -65,6 +67,15 @@ public class ExtendedSequenceNumber implements Comparable SENTINEL_VALUES = Collections.unmodifiableSet( + Arrays.stream(SentinelCheckpoint.values()).map(SentinelCheckpoint::name).collect(Collectors.toSet())); + /** * Construct an ExtendedSequenceNumber. The sub-sequence number defaults to * 0. @@ -87,7 +98,7 @@ public class ExtendedSequenceNumber implements Comparable= 0) { - sb.append("SubsequenceNumber: " + subSequenceNumber()); + sb.append("SubsequenceNumber: ").append(subSequenceNumber()); } - sb.append("}"); + sb.append('}'); return sb.toString(); } - @Override - public int hashCode() { - final int prime = 31; - final int shift = 32; - int hashCode = 1; - hashCode = prime * hashCode + ((sequenceNumber == null) ? 0 : sequenceNumber.hashCode()); - hashCode = prime * hashCode + ((subSequenceNumber < 0) - ? 0 - : (int) (subSequenceNumber ^ (subSequenceNumber >>> shift))); - return hashCode; - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - - if (!(obj instanceof ExtendedSequenceNumber)) { - return false; - } - ExtendedSequenceNumber other = (ExtendedSequenceNumber) obj; - - if (!sequenceNumber.equals(other.sequenceNumber())) { - return false; - } - return subSequenceNumber == other.subSequenceNumber(); - } - /** * Sequence numbers are converted, sentinels are given a value of -1. Note this method is only used after special * logic associated with SHARD_END and the case of comparing two sentinel values has already passed, so we map @@ -217,30 +195,23 @@ public class ExtendedSequenceNumber implements Comparable Date: Mon, 6 Mar 2023 17:05:34 -0500 Subject: [PATCH 26/88] Added more logging in `Scheduler` w.r.t. `StreamConfig`s. (#1057) --- .../amazon/kinesis/coordinator/Scheduler.java | 33 +++++++------- .../kinesis/coordinator/SchedulerTest.java | 44 +++++++++---------- 2 files changed, 36 insertions(+), 41 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 743be28a..e95ddb6f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -224,6 +224,7 @@ public class Scheduler implements Runnable { this.formerStreamsLeasesDeletionStrategy = streamTracker.formerStreamsLeasesDeletionStrategy(); streamTracker.streamConfigList().forEach( sc -> currentStreamConfigMap.put(sc.streamIdentifier(), sc)); + log.info("Initial state: {}", currentStreamConfigMap.values()); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); @@ -449,18 +450,15 @@ public class Scheduler implements Runnable { final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, MULTI_STREAM_TRACKER); try { - final Map newStreamConfigMap = streamTracker.streamConfigList() - .stream().collect(Collectors.toMap(StreamConfig::streamIdentifier, Function.identity())); - - List leases; - // This is done to ensure that we clean up the stale streams lingering in the lease table. if (!leasesSyncedOnAppInit && isMultiStreamMode) { - leases = fetchMultiStreamLeases(); + final List leases = fetchMultiStreamLeases(); syncStreamsFromLeaseTableOnAppInit(leases); leasesSyncedOnAppInit = true; } + final Map newStreamConfigMap = streamTracker.streamConfigList() + .stream().collect(Collectors.toMap(StreamConfig::streamIdentifier, Function.identity())); // For new streams discovered, do a shard sync and update the currentStreamConfigMap for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { @@ -471,9 +469,7 @@ public class Scheduler implements Runnable { currentStreamConfigMap.put(streamIdentifier, streamConfig); streamsSynced.add(streamIdentifier); } else { - if (log.isDebugEnabled()) { - log.debug(streamIdentifier + " is already being processed - skipping shard sync."); - } + log.debug("{} is already being processed - skipping shard sync.", streamIdentifier); } } @@ -536,7 +532,7 @@ public class Scheduler implements Runnable { // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and // the streamIdentifiersForLeaseCleanup are not present in the latest snapshot. final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors - .partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), Collectors.toSet())); + .partitioningBy(newStreamConfigMap::containsKey, Collectors.toSet())); final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier -> Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); final Set deletedStreamsLeases = deleteMultiStreamLeases(staleStreamIdsToBeDeleted); @@ -572,14 +568,14 @@ public class Scheduler implements Runnable { } @VisibleForTesting void syncStreamsFromLeaseTableOnAppInit(List leases) { - final Set streamIdentifiers = leases.stream() + leases.stream() .map(lease -> StreamIdentifier.multiStreamInstance(lease.streamIdentifier())) - .collect(Collectors.toSet()); - for (StreamIdentifier streamIdentifier : streamIdentifiers) { - if (!currentStreamConfigMap.containsKey(streamIdentifier)) { - currentStreamConfigMap.put(streamIdentifier, streamTracker.createStreamConfig(streamIdentifier)); - } - } + .filter(streamIdentifier -> !currentStreamConfigMap.containsKey(streamIdentifier)) + .forEach(streamIdentifier -> { + final StreamConfig streamConfig = streamTracker.createStreamConfig(streamIdentifier); + currentStreamConfigMap.put(streamIdentifier, streamConfig); + log.info("Cached {}", streamConfig); + }); } private List fetchMultiStreamLeases() @@ -897,6 +893,7 @@ public class Scheduler implements Runnable { StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); if (streamConfig == null) { streamConfig = streamTracker.createStreamConfig(streamIdentifier); + log.info("Created orphan {}", streamConfig); } Validate.notNull(streamConfig, "StreamConfig should not be null"); RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, streamConfig, metricsFactory); @@ -993,7 +990,7 @@ public class Scheduler implements Runnable { @NoArgsConstructor(access = AccessLevel.PRIVATE) private static class SchedulerLog { - private long reportIntervalMillis = TimeUnit.MINUTES.toMillis(1); + private final long reportIntervalMillis = TimeUnit.MINUTES.toMillis(1); private long nextReportTime = System.currentTimeMillis() + reportIntervalMillis; private boolean infoReporting; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index aa9f8412..90b63477 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -48,6 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.RejectedExecutionException; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -317,7 +318,7 @@ public class SchedulerTest { } @Test - public final void testMultiStreamInitialization() throws ProvisionedThroughputException, DependencyException { + public final void testMultiStreamInitialization() { retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) .retrievalFactory(retrievalFactory); leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, @@ -325,9 +326,9 @@ public class SchedulerTest { scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); scheduler.initialize(); - shardDetectorMap.values().stream() + shardDetectorMap.values() .forEach(shardDetector -> verify(shardDetector, times(1)).listShards()); - shardSyncTaskManagerMap.values().stream() + shardSyncTaskManagerMap.values() .forEach(shardSyncTM -> verify(shardSyncTM, times(1)).hierarchicalShardSyncer()); } @@ -343,17 +344,16 @@ public class SchedulerTest { // Note : As of today we retry for all streams in the next attempt. Hence the retry for each stream will vary. // At the least we expect 2 retries for each stream. Since there are 4 streams, we expect at most // the number of calls to be 5. - shardDetectorMap.values().stream() - .forEach(shardDetector -> verify(shardDetector, atLeast(2)).listShards()); - shardDetectorMap.values().stream() - .forEach(shardDetector -> verify(shardDetector, atMost(5)).listShards()); - shardSyncTaskManagerMap.values().stream() - .forEach(shardSyncTM -> verify(shardSyncTM, atLeast(2)).hierarchicalShardSyncer()); - shardSyncTaskManagerMap.values().stream() - .forEach(shardSyncTM -> verify(shardSyncTM, atMost(5)).hierarchicalShardSyncer()); + shardDetectorMap.values().forEach(shardDetector -> { + verify(shardDetector, atLeast(2)).listShards(); + verify(shardDetector, atMost(5)).listShards(); + }); + shardSyncTaskManagerMap.values().forEach(shardSyncTM -> { + verify(shardSyncTM, atLeast(2)).hierarchicalShardSyncer(); + verify(shardSyncTM, atMost(5)).hierarchicalShardSyncer(); + }); } - @Test public final void testMultiStreamConsumersAreBuiltOncePerAccountStreamShard() throws KinesisClientLibException { final String shardId = "shardId-000000000000"; @@ -385,13 +385,12 @@ public class SchedulerTest { schedulerSpy.runProcessLoop(); schedulerSpy.runProcessLoop(); - initialShardInfo.stream().forEach( + initialShardInfo.forEach( shardInfo -> verify(schedulerSpy).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), same(leaseCleanupManager))); - firstShardInfo.stream().forEach( + firstShardInfo.forEach( shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager))); - secondShardInfo.stream().forEach( + secondShardInfo.forEach( shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager))); - } @Test @@ -415,7 +414,7 @@ public class SchedulerTest { when(scheduler.shouldSyncStreamsNow()).thenReturn(true); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Assert.assertTrue("SyncedStreams should be empty", syncedStreams.isEmpty()); - Assert.assertEquals(new HashSet(streamConfigList1), new HashSet(scheduler.currentStreamConfigMap().values())); + assertEquals(new HashSet<>(streamConfigList1), new HashSet<>(scheduler.currentStreamConfigMap().values())); } @Test @@ -447,8 +446,7 @@ public class SchedulerTest { } @Test - public final void testMultiStreamSyncFromTableDefaultInitPos() - throws DependencyException, ProvisionedThroughputException, InvalidStateException { + public final void testMultiStreamSyncFromTableDefaultInitPos() { // Streams in lease table but not tracked by multiStreamTracker List leasesInTable = IntStream.range(1, 3).mapToObj(streamId -> new MultiStreamLease() .streamIdentifier( @@ -474,13 +472,12 @@ public class SchedulerTest { metricsConfig, processorConfig, retrievalConfig); scheduler.syncStreamsFromLeaseTableOnAppInit(leasesInTable); Map expectedConfigMap = expectedConfig.stream().collect(Collectors.toMap( - sc -> sc.streamIdentifier(), sc -> sc)); + StreamConfig::streamIdentifier, Function.identity())); Assert.assertEquals(expectedConfigMap, scheduler.currentStreamConfigMap()); } @Test - public final void testMultiStreamSyncFromTableCustomInitPos() - throws DependencyException, ProvisionedThroughputException, InvalidStateException { + public final void testMultiStreamSyncFromTableCustomInitPos() { Date testTimeStamp = new Date(); // Streams in lease table but not tracked by multiStreamTracker @@ -725,7 +722,8 @@ public class SchedulerTest { testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false); } - private final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion, + private void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately( + boolean expectPendingStreamsForDeletion, boolean onlyStreamsNoLeasesDeletion) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( From 43d43653d0f247d8a21a9e500fc2fdbc08836c0a Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Tue, 7 Mar 2023 15:36:25 -0500 Subject: [PATCH 27/88] Documentation: added `
` tags so fixed-format
 diagrams aren't garbled. (#1058)

No functional change.
---
 .../leases/HierarchicalShardSyncer.java       |  18 +-
 .../leases/HierarchicalShardSyncerTest.java   | 184 +++++++++++++-----
 2 files changed, 141 insertions(+), 61 deletions(-)

diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java
index e44125a5..175d62ec 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java
@@ -855,22 +855,20 @@ public class HierarchicalShardSyncer {
          * * 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)
-         *
+         * 
+ * Assuming current leases are (4, 5, 7), new leases to create for an initial position are: + *
    + *
  • LATEST: (6)
  • + *
  • TRIM_HORIZON: (0, 1)
  • + *
  • AT_TIMESTAMP(epoch=200): (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 diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index c390987c..c8ef05ba 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertThat; import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -73,7 +72,6 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import static software.amazon.kinesis.leases.HierarchicalShardSyncer.MemoizationContext; @RunWith(MockitoJUnitRunner.class) -// CHECKSTYLE:IGNORE JavaNCSS FOR NEXT 800 LINES public class HierarchicalShardSyncerTest { private static final InitialPositionInStreamExtended INITIAL_POSITION_LATEST = InitialPositionInStreamExtended .newInitialPosition(InitialPositionInStream.LATEST); @@ -320,7 +318,6 @@ public class HierarchicalShardSyncerTest { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); - } @Test @@ -355,7 +352,6 @@ public class HierarchicalShardSyncerTest { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); - } private List toMultiStreamLeaseList(List shardIdBasedLeases) { @@ -460,7 +456,6 @@ public class HierarchicalShardSyncerTest { final Set expectedShardIds = new HashSet<>(); final List requestLeases = leaseCaptor.getAllValues(); - final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); validateHashRangeinLease(requestLeases); @@ -472,7 +467,8 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } - /* + /** + *

      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -481,6 +477,7 @@ public class HierarchicalShardSyncerTest {
      *    8    4 9  10 - shards from epoch 206 (open - no ending sequenceNumber)
      * Initial position: TRIM_HORIZON
      * Leases to create: (0, 1, 2, 3, 4, 5)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithEmptyLeaseTable() throws Exception { @@ -490,7 +487,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -499,6 +497,7 @@ public class HierarchicalShardSyncerTest {
      *    8    4 9  10 - shards from epoch 206 (open - no ending sequenceNumber)
      * Initial position: AT_TIMESTAMP(1000)
      * Leases to create: (8, 4, 9, 10)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable1() throws Exception { @@ -508,7 +507,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -517,6 +517,7 @@ public class HierarchicalShardSyncerTest {
      *    8    4 9  10 - shards from epoch 206 (open - no ending sequenceNumber)
      * Initial position: AT_TIMESTAMP(200)
      * Leases to create: (6, 7, 4, 5)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable2() throws Exception { @@ -528,7 +529,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -537,6 +539,7 @@ public class HierarchicalShardSyncerTest {
      *    8    4 9  10 - shards from epoch 206 (open - no ending sequenceNumber)
      * Initial position: LATEST
      * Leases to create: (8, 4, 9, 10)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() throws Exception { @@ -546,7 +549,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -556,6 +560,7 @@ public class HierarchicalShardSyncerTest {
      * Missing leases: (0, 6, 8)
      * Initial position: TRIM_HORIZON
      * Leases to create: (0)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithPartialLeaseTable() throws Exception { @@ -571,7 +576,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate, existingLeases); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -581,6 +587,7 @@ public class HierarchicalShardSyncerTest {
      * Missing leases: (0, 6, 8)
      * Initial position: AT_TIMESTAMP(1000)
      * Leases to create: (0)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable1() throws Exception { @@ -596,7 +603,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate, existingLeases); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -606,6 +614,7 @@ public class HierarchicalShardSyncerTest {
      * Missing leases: (0, 6, 8)
      * Initial position: AT_TIMESTAMP(200)
      * Leases to create: (0)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable2() throws Exception { @@ -623,7 +632,8 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate, existingLeases); } - /* + /** + *
      * Shard structure (each level depicts a stream segment):
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
@@ -633,6 +643,7 @@ public class HierarchicalShardSyncerTest {
      * Missing leases: (0, 6, 8)
      * Initial position: LATEST
      * Leases to create: (0)
+     * 
*/ @Test public void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() throws Exception { @@ -835,9 +846,6 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); - final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); assertThat(deleteLeases.size(), equalTo(0)); @@ -1074,7 +1082,6 @@ public class HierarchicalShardSyncerTest { final InitialPositionInStreamExtended initialPosition, final Set expectedLeaseKeys, final List existingLeases) throws Exception { - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); @@ -1168,6 +1175,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1176,6 +1184,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (3, 4, 5)
      * Initial position: LATEST
      * Expected leases: (2, 6)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange1() { @@ -1190,6 +1199,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1198,7 +1208,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 5, 7)
      * Initial position: LATEST
      * Expected leases: (6)
-     *
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange2() { @@ -1212,6 +1222,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1220,6 +1231,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (2, 6)
      * Initial position: LATEST
      * Expected leases: (3, 4, 9, 10)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange3() { @@ -1236,6 +1248,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1244,6 +1257,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 9, 10)
      * Initial position: LATEST
      * Expected leases: (8)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange4() { @@ -1256,6 +1270,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors + *
      * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is
      * epochs):     0      1  2  3  - shards till
      *            /   \    |  \ /
@@ -1266,6 +1281,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (9, 10)
      * Initial position: LATEST
      * Expected leases: (1, 6, 7, 8)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestC_PartialHashRange5() { @@ -1282,6 +1298,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1290,6 +1307,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 5, 6, 7)
      * Initial position: LATEST
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRange() { @@ -1302,6 +1320,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1310,6 +1329,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 2, 3, 4, 5, 6, 7)
      * Initial position: LATEST
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeWithoutGC() { @@ -1323,6 +1343,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1331,6 +1352,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: empty set
      * Initial position: LATEST
      * Expected leases: (4, 8, 9, 10)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_EmptyLeaseTable() { @@ -1347,6 +1369,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1355,6 +1378,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 4, 7, 9, 10)
      * Initial position: LATEST
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeAcrossDifferentEpochs() { @@ -1365,7 +1389,8 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1376,17 +1401,19 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (6)
      * Initial position: LATEST
      * Expected leases: (7)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_PartialHashRange() { final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); + final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-6"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1397,16 +1424,18 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (5)
      * Initial position: LATEST
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRange() { final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-5"); final Map expectedNoNewLeases = Collections.emptyMap(); assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1417,6 +1446,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 2, 3, 4, 5)
      * Initial position: LATEST
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRangeWithoutGC() { @@ -1427,7 +1457,8 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1438,6 +1469,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: empty set
      * Initial position: LATEST
      * Expected leases: (9, 10)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_EmptyLeaseTable() { @@ -1452,6 +1484,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1460,6 +1493,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (3, 4, 5)
      * Initial position: TRIM_HORIZON
      * Expected leases: (0, 1, 2)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange1() { @@ -1475,6 +1509,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1483,6 +1518,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 5, 7)
      * Initial position: TRIM_HORIZON
      * Expected leases: (0, 1)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange2() { @@ -1497,6 +1533,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1505,6 +1542,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (2, 6)
      * Initial position: TRIM_HORIZON
      * Expected leases: (3, 4, 5)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange3() { @@ -1520,6 +1558,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1528,6 +1567,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 9, 10)
      * Initial position: TRIM_HORIZON
      * Expected leases: (0, 1, 2, 3)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange4() { @@ -1544,6 +1584,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1552,6 +1593,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 5, 6, 7)
      * Initial position: TRIM_HORIZON
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRange() { @@ -1564,6 +1606,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1572,6 +1615,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 2, 3, 4, 5, 6, 7)
      * Initial position: TRIM_HORIZON
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeWithoutGC() { @@ -1585,6 +1629,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1593,6 +1638,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: empty set
      * Initial position: TRIM_HORIZON
      * Expected leases: (0, 1, 2, 3, 4, 5)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_EmptyLeaseTable() { @@ -1611,6 +1657,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1619,6 +1666,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 4, 7, 9, 10)
      * Initial position: TRIM_HORIZON
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeAcrossDifferentEpochs() { @@ -1629,7 +1677,8 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1640,6 +1689,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (6)
      * Initial position: TRIM_HORIZON
      * Expected leases: (7)
+     * 
*/ // TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases // @Test @@ -1651,7 +1701,8 @@ public class HierarchicalShardSyncerTest { // assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); // } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1662,16 +1713,18 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (5)
      * Initial position: TRIM_HORIZON
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRange() { final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-5"); final Map expectedNoNewLeases = Collections.emptyMap(); assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1682,6 +1735,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 2, 3, 4, 5)
      * Initial position: TRIM_HORIZON
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRangeWithoutGC() { @@ -1692,7 +1746,8 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1703,6 +1758,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: empty set
      * Initial position: TRIM_HORIZON
      * Expected leases: (0, 1)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonB_EmptyLeaseTable() { @@ -1717,6 +1773,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1725,6 +1782,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (3, 4, 5)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (0, 1, 2)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange1() { @@ -1740,6 +1798,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1748,6 +1807,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 5, 7)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (0, 1)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange2() { @@ -1762,6 +1822,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1770,6 +1831,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (2, 6)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (3, 4, 5)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange3() { @@ -1785,6 +1847,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1793,6 +1856,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 9, 10)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (0, 1, 2, 3)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange4() { @@ -1809,6 +1873,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1817,6 +1882,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (4, 5, 6, 7)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRange() { @@ -1829,6 +1895,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1837,6 +1904,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 2, 3, 4, 5, 6, 7)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeWithoutGC() { @@ -1850,6 +1918,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1858,6 +1927,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: empty set
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (0, 1, 2, 3, 4, 5)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_EmptyLeaseTable() { @@ -1876,6 +1946,7 @@ public class HierarchicalShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): + *
      * 0 1 2 3 4   5- shards till epoch 102
      * \ / \ / |   |
      *  6   7  4   5- shards from epoch 103 - 205
@@ -1884,6 +1955,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 4, 7, 9, 10)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeAcrossDifferentEpochs() { @@ -1894,7 +1966,8 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1905,6 +1978,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (6)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (7)
+     * 
*/ // TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases // @Test @@ -1916,7 +1990,8 @@ public class HierarchicalShardSyncerTest { // assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); // } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1927,16 +2002,18 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (5)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRange() { final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-5"); final Map expectedNoNewLeases = Collections.emptyMap(); assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1947,6 +2024,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: (0, 1, 2, 3, 4, 5)
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: empty set
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRangeWithoutGC() { @@ -1957,7 +2035,8 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); } - /* + /** + *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
      * \ / \ / \ /
@@ -1968,6 +2047,7 @@ public class HierarchicalShardSyncerTest {
      * Current leases: empty set
      * Initial position: AT_TIMESTAMP(1000)
      * Expected leases: (0, 1)
+     * 
*/ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_EmptyLeaseTable() { @@ -2005,7 +2085,8 @@ public class HierarchicalShardSyncerTest { } } - /* + /** + *
      * Helper method to construct a shard list for graph A. Graph A is defined below. Shard structure (y-axis is
      * epochs): 0 1 2 3 4   5- shards till
      *          \ / \ / |   |
@@ -2013,6 +2094,7 @@ public class HierarchicalShardSyncerTest {
      *            \ /   |  /\
      *             8    4 9 10 -
      * shards from epoch 206 (open - no ending sequenceNumber)
+     * 
*/ private List constructShardListForGraphA() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); @@ -2075,7 +2157,8 @@ public class HierarchicalShardSyncerTest { throw new RuntimeException("Unsupported initial position " + initialPosition); } - /* + /** + *
      * Helper method to get expected shards for Graph A based on initial position in stream. Shard structure (y-axis is
      * epochs): 0 1 2 3 4   5- shards till
      *          \ / \ / |   |
@@ -2083,6 +2166,7 @@ public class HierarchicalShardSyncerTest {
      *            \ /   |  /\
      *             8    4 9 10 -
      * shards from epoch 206 (open - no ending sequenceNumber)
+     * 
*/ private Set getExpectedLeasesForGraphA(List shards, ExtendedSequenceNumber sequenceNumber, @@ -2097,15 +2181,17 @@ public class HierarchicalShardSyncerTest { return new HashSet<>(createLeasesFromShards(filteredShards, sequenceNumber, null)); } -// /* -// * Helper method to construct a shard list for graph B. Graph B is defined below. -// * Shard structure (x-axis is epochs): -// * 0 3 6 9 -// * \ / \ / \ / -// * 2 5 8 -// * / \ / \ / \ -// * 1 4 7 10 -// */ + /** + * Helper method to construct a shard list for graph B. Graph B is defined below. + * Shard structure (x-axis is epochs): + *
+     * 0  3   6   9
+     * \ / \ / \ /
+     *  2   5   8
+     * / \ / \ / \
+     * 1  4   7  10
+     * 
+ */ private List constructShardListForGraphB() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("1000", "1049"); final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("1050", "1099"); @@ -2133,6 +2219,7 @@ public class HierarchicalShardSyncerTest { } /** + *
      * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is
      * epochs):     0      1  2  3  - shards till
      *            /   \    |  \ /
@@ -2140,6 +2227,7 @@ public class HierarchicalShardSyncerTest {
      *          / \   / \  |   |
      *         7   8 9  10 1   6
      * shards from epoch 206 (open - no ending sequenceNumber)
+     * 
*/ private List constructShardListForGraphC() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); @@ -2241,7 +2329,6 @@ public class HierarchicalShardSyncerTest { /** * Tests that when reading from TIP, we use the AT_LATEST shard filter. - * @throws Exception */ @Test public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtLatest() throws Exception { @@ -2251,7 +2338,6 @@ public class HierarchicalShardSyncerTest { /** * Tests that when reading from TRIM, we use the TRIM_HORIZON shard filter. - * @throws Exception */ @Test public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTrimHorizon() throws Exception { @@ -2261,7 +2347,6 @@ public class HierarchicalShardSyncerTest { /** * Tests that when reading from AT_TIMESTAMP, we use the AT_TIMESTAMP shard filter. - * @throws Exception */ @Test public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTimestamp() throws Exception { @@ -2309,7 +2394,6 @@ public class HierarchicalShardSyncerTest { /** * Tries to boostrap empty lease table. Verifies that if we fail to get a complete hash range of shards after three * retries, we fast fail and throw an exception. - * @throws Exception */ @Test(expected = KinesisClientLibIOException.class) public void testEmptyLeaseTableThrowsExceptionWhenHashRangeIsStillIncompleteAfterRetries() throws Exception { @@ -2334,7 +2418,6 @@ public class HierarchicalShardSyncerTest { /** * Tries to bootstrap an empty lease table. Verifies that after getting an incomplete hash range of shards two times * and a complete hash range the final time, we create the leases. - * @throws Exception */ @Test public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRangeAfterTwoRetries() throws Exception { @@ -2362,7 +2445,6 @@ public class HierarchicalShardSyncerTest { /** * Tries to bootstrap an empty lease table. Verifies that leases are created when we have a complete hash range of shards. - * @throws Exception */ @Test public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRange() throws Exception { From 27b166c5aab9a7c2b75b26a02a3987dec9bf5599 Mon Sep 17 00:00:00 2001 From: Abhit Sawwalakhe <31434088+abhit17@users.noreply.github.com> Date: Wed, 8 Mar 2023 13:39:35 -0800 Subject: [PATCH 28/88] Clean up in-memory state of deleted kinesis stream in MultiStreamMode (#1056) Co-authored-by: Abhit Sawwalakhe --- .../DeletedStreamListProvider.java | 38 +++++++ .../amazon/kinesis/coordinator/Scheduler.java | 23 +++- .../leases/HierarchicalShardSyncer.java | 27 ++++- .../kinesis/leases/KinesisShardDetector.java | 33 +++++- .../leases/LeaseManagementFactory.java | 6 ++ .../amazon/kinesis/leases/ShardDetector.java | 10 ++ .../DynamoDBLeaseManagementFactory.java | 19 +++- .../kinesis/coordinator/SchedulerTest.java | 100 ++++++++++++++++-- .../leases/HierarchicalShardSyncerTest.java | 92 +++++++++++----- 9 files changed, 296 insertions(+), 52 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeletedStreamListProvider.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeletedStreamListProvider.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeletedStreamListProvider.java new file mode 100644 index 00000000..d0d332d9 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeletedStreamListProvider.java @@ -0,0 +1,38 @@ +package software.amazon.kinesis.coordinator; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import lombok.extern.slf4j.Slf4j; + +import software.amazon.kinesis.common.StreamIdentifier; + +/** + * This class is used for storing in-memory set of streams which are no longer existing (deleted) and needs to be + * cleaned up from KCL's in memory state. + */ +@Slf4j +public class DeletedStreamListProvider { + + private final Set deletedStreams; + + public DeletedStreamListProvider() { + deletedStreams = ConcurrentHashMap.newKeySet(); + } + + public void add(StreamIdentifier streamIdentifier) { + log.info("Added {}", streamIdentifier); + deletedStreams.add(streamIdentifier); + } + + /** + * Method returns and empties the current set of streams + * @return set of deleted Streams + */ + public Set purgeAllDeletedStream() { + final Set response = new HashSet<>(deletedStreams); + deletedStreams.removeAll(response); + return response; + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index e95ddb6f..dacb7ba1 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -116,6 +116,7 @@ public class Scheduler implements Runnable { private static final String ACTIVE_STREAMS_COUNT = "ActiveStreams.Count"; private static final String PENDING_STREAMS_DELETION_COUNT = "StreamsPendingDeletion.Count"; private static final String DELETED_STREAMS_COUNT = "DeletedStreams.Count"; + private static final String NON_EXISTING_STREAM_DELETE_COUNT = "NonExistingStreamDelete.Count"; private final SchedulerLog slog = new SchedulerLog(); @@ -166,6 +167,8 @@ public class Scheduler implements Runnable { private final LeaseCleanupManager leaseCleanupManager; private final SchemaRegistryDecoder schemaRegistryDecoder; + private final DeletedStreamListProvider deletedStreamListProvider; + // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. private final ConcurrentMap shardInfoShardConsumerMap = new ConcurrentHashMap<>(); @@ -251,9 +254,10 @@ public class Scheduler implements Runnable { this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService(); this.diagnosticEventFactory = diagnosticEventFactory; this.diagnosticEventHandler = new DiagnosticEventLogger(); + this.deletedStreamListProvider = new DeletedStreamListProvider(); this.shardSyncTaskManagerProvider = streamConfig -> this.leaseManagementConfig .leaseManagementFactory(leaseSerializer, isMultiStreamMode) - .createShardSyncTaskManager(this.metricsFactory, streamConfig); + .createShardSyncTaskManager(this.metricsFactory, streamConfig, this.deletedStreamListProvider); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -535,6 +539,19 @@ public class Scheduler implements Runnable { .partitioningBy(newStreamConfigMap::containsKey, Collectors.toSet())); final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier -> Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); + // These are the streams which are deleted in Kinesis and we encounter resource not found during + // shardSyncTask. This is applicable in MultiStreamMode only, in case of SingleStreamMode, store will + // not have any data. + // Filter streams based on newStreamConfigMap so that we don't override input to KCL in any case. + final Set deletedStreamSet = this.deletedStreamListProvider + .purgeAllDeletedStream() + .stream() + .filter(streamIdentifier -> !newStreamConfigMap.containsKey(streamIdentifier)) + .collect(Collectors.toSet()); + if (deletedStreamSet.size() > 0) { + log.info("Stale streams to delete: {}", deletedStreamSet); + staleStreamIdsToBeDeleted.addAll(deletedStreamSet); + } final Set deletedStreamsLeases = deleteMultiStreamLeases(staleStreamIdsToBeDeleted); streamsSynced.addAll(deletedStreamsLeases); @@ -554,6 +571,8 @@ public class Scheduler implements Runnable { MetricsUtil.addCount(metricsScope, ACTIVE_STREAMS_COUNT, newStreamConfigMap.size(), MetricsLevel.SUMMARY); MetricsUtil.addCount(metricsScope, PENDING_STREAMS_DELETION_COUNT, staleStreamDeletionMap.size(), MetricsLevel.SUMMARY); + MetricsUtil.addCount(metricsScope, NON_EXISTING_STREAM_DELETE_COUNT, deletedStreamSet.size(), + MetricsLevel.SUMMARY); MetricsUtil.addCount(metricsScope, DELETED_STREAMS_COUNT, deletedStreamsLeases.size(), MetricsLevel.SUMMARY); } finally { MetricsUtil.endScope(metricsScope); @@ -594,7 +613,7 @@ public class Scheduler implements Runnable { if (streamIdentifiers.isEmpty()) { return Collections.emptySet(); } - + log.info("Deleting streams: {}", streamIdentifiers); final Set streamsSynced = new HashSet<>(); final List leases = fetchMultiStreamLeases(); final Map> streamIdToShardsMap = leases.stream().collect( diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index 175d62ec..aafbfcff 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -17,6 +17,7 @@ package software.amazon.kinesis.leases; import java.io.Serializable; import java.math.BigInteger; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -39,6 +40,7 @@ import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.services.kinesis.model.ShardFilterType; @@ -47,6 +49,7 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.coordinator.DeletedStreamListProvider; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; @@ -56,6 +59,7 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import static java.util.Objects.nonNull; import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRange; /** @@ -72,6 +76,8 @@ public class HierarchicalShardSyncer { private final String streamIdentifier; + private final DeletedStreamListProvider deletedStreamListProvider; + private static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); private static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); private static final int retriesForCompleteHashRange = 3; @@ -79,13 +85,17 @@ public class HierarchicalShardSyncer { private static final long DELAY_BETWEEN_LIST_SHARDS_MILLIS = 1000; public HierarchicalShardSyncer() { - isMultiStreamMode = false; - streamIdentifier = "SingleStreamMode"; + this(false, "SingleStreamMode"); } public HierarchicalShardSyncer(final boolean isMultiStreamMode, final String streamIdentifier) { + this(isMultiStreamMode, streamIdentifier, null); + } + + public HierarchicalShardSyncer(final boolean isMultiStreamMode, final String streamIdentifier, final DeletedStreamListProvider deletedStreamListProvider) { this.isMultiStreamMode = isMultiStreamMode; this.streamIdentifier = streamIdentifier; + this.deletedStreamListProvider = deletedStreamListProvider; } private static final BiFunction shardIdFromLeaseDeducer = @@ -279,8 +289,17 @@ public class HierarchicalShardSyncer { + retriesForCompleteHashRange + " retries."); } - private static List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { - final Optional> shards = Optional.of(shardDetector.listShards()); + private List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { + // Fallback to existing behavior for backward compatibility + List shardList = Collections.emptyList(); + try { + shardList = shardDetector.listShardsWithoutConsumingResourceNotFoundException(); + } catch (ResourceNotFoundException e) { + if (nonNull(this.deletedStreamListProvider) && isMultiStreamMode) { + deletedStreamListProvider.add(StreamIdentifier.multiStreamInstance(streamIdentifier)); + } + } + final Optional> shards = Optional.of(shardList); return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream " + shardDetector.streamIdentifier().streamName() + " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java index 0c3de1bd..9a44a553 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -96,6 +96,8 @@ public class KinesisShardDetector implements ShardDetector { @Getter(AccessLevel.PACKAGE) private final AtomicInteger cacheMisses = new AtomicInteger(0); + private static final Boolean THROW_RESOURCE_NOT_FOUND_EXCEPTION = true; + @Deprecated public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload, @@ -175,15 +177,26 @@ public class KinesisShardDetector implements ShardDetector { return listShardsWithFilter(null); } + @Override + @Synchronized + public List listShardsWithoutConsumingResourceNotFoundException() { + return listShardsWithFilterInternal(null, THROW_RESOURCE_NOT_FOUND_EXCEPTION); + } + @Override @Synchronized public List listShardsWithFilter(ShardFilter shardFilter) { + return listShardsWithFilterInternal(shardFilter, !THROW_RESOURCE_NOT_FOUND_EXCEPTION); + } + + private List listShardsWithFilterInternal(ShardFilter shardFilter, + boolean shouldPropagateResourceNotFoundException) { final List shards = new ArrayList<>(); ListShardsResponse result; String nextToken = null; do { - result = listShards(shardFilter, nextToken); + result = listShards(shardFilter, nextToken, shouldPropagateResourceNotFoundException); if (result == null) { /* @@ -201,7 +214,12 @@ public class KinesisShardDetector implements ShardDetector { return shards; } - private ListShardsResponse listShards(ShardFilter shardFilter, final String nextToken) { + /** + * @param shouldPropagateResourceNotFoundException : used to determine if ResourceNotFoundException should be + * handled by method and return Empty list or propagate the exception. + */ + private ListShardsResponse listShards(ShardFilter shardFilter, final String nextToken, + final boolean shouldPropagateResourceNotFoundException) { ListShardsRequest.Builder builder = KinesisRequestsBuilder.listShardsRequestBuilder(); if (StringUtils.isEmpty(nextToken)) { builder = builder.streamName(streamIdentifier.streamName()).shardFilter(shardFilter); @@ -243,9 +261,14 @@ public class KinesisShardDetector implements ShardDetector { } catch (ResourceNotFoundException e) { log.warn("Got ResourceNotFoundException when fetching shard list for {}. Stream no longer exists.", streamIdentifier.streamName()); - return ListShardsResponse.builder().shards(Collections.emptyList()) - .nextToken(null) - .build(); + if (shouldPropagateResourceNotFoundException) { + throw e; + } + return ListShardsResponse.builder() + .shards(Collections.emptyList()) + .nextToken(null) + .build(); + } catch (TimeoutException te) { throw new RuntimeException(te); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java index ecf9b390..9f2e5f94 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.leases; import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.coordinator.DeletedStreamListProvider; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.metrics.MetricsFactory; @@ -31,6 +32,11 @@ public interface LeaseManagementFactory { throw new UnsupportedOperationException(); } + default ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig, + DeletedStreamListProvider deletedStreamListProvider) { + throw new UnsupportedOperationException("createShardSyncTaskManager method not implemented"); + } + DynamoDBLeaseRefresher createLeaseRefresher(); ShardDetector createShardDetector(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java index 62b93855..32514eb5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java @@ -46,6 +46,16 @@ public interface ShardDetector { */ List listShards(); + /** + * This method behaves exactly similar to listShards except the fact that this does not consume and throw + * ResourceNotFoundException instead of returning empty list. + * + * @return Shards + */ + default List listShardsWithoutConsumingResourceNotFoundException() { + throw new UnsupportedOperationException("listShardsWithoutConsumingResourceNotFoundException not implemented"); + } + /** * List shards with shard filter. * diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java index ad1a2300..6bf2ff39 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -29,6 +29,7 @@ import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.LeaseCleanupConfig; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.coordinator.DeletedStreamListProvider; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.KinesisShardDetector; import software.amazon.kinesis.leases.LeaseCleanupManager; @@ -504,6 +505,20 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { */ @Override public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { + return createShardSyncTaskManager(metricsFactory, streamConfig, null); + } + + /** + * Create ShardSyncTaskManager from the streamConfig passed + * + * @param metricsFactory - factory to get metrics object + * @param streamConfig - streamConfig for which ShardSyncTaskManager needs to be created + * @param deletedStreamListProvider - store for capturing the streams which are deleted in kinesis + * @return ShardSyncTaskManager + */ + @Override + public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig, + DeletedStreamListProvider deletedStreamListProvider) { return new ShardSyncTaskManager(this.createShardDetector(streamConfig), this.createLeaseRefresher(), streamConfig.initialPositionInStreamExtended(), @@ -511,10 +526,12 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, executorService, - new HierarchicalShardSyncer(isMultiStreamMode, streamConfig.streamIdentifier().toString()), + new HierarchicalShardSyncer(isMultiStreamMode, streamConfig.streamIdentifier().toString(), + deletedStreamListProvider), metricsFactory); } + @Override public DynamoDBLeaseRefresher createLeaseRefresher() { return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, leaseSerializer, consistentReads, diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 90b63477..af0755a3 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -39,6 +39,7 @@ import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrat import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashMap; @@ -726,16 +727,8 @@ public class SchedulerTest { boolean expectPendingStreamsForDeletion, boolean onlyStreamsNoLeasesDeletion) throws DependencyException, ProvisionedThroughputException, InvalidStateException { - List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( - StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), - InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) - .collect(Collectors.toCollection(LinkedList::new)); - List streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( - StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), - InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) - .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList1 = createDummyStreamConfigList(1,5); + List streamConfigList2 = createDummyStreamConfigList(3,7); retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) .retrievalFactory(retrievalFactory); when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); @@ -782,6 +775,91 @@ public class SchedulerTest { scheduler.staleStreamDeletionMap().keySet()); } + + @Test + public void testKinesisStaleDeletedStreamCleanup() throws ProvisionedThroughputException, InvalidStateException, DependencyException { + List streamConfigList1 = createDummyStreamConfigList(1,6); + List streamConfigList2 = createDummyStreamConfigList(1,4); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + + prepareForStaleDeletedStreamCleanupTests(); + + // when KCL starts it starts with tracking 5 stream + assertEquals(Sets.newHashSet(streamConfigList1), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + assertEquals(0, scheduler.staleStreamDeletionMap().size()); + + // 2 Streams are no longer needed to be consumed + Set syncedStreams1 = scheduler.checkAndSyncStreamShardsAndLeases(); + assertEquals(Sets.newHashSet(streamConfigList1), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + assertEquals(createDummyStreamConfigList(4, 6).stream() + .map(StreamConfig::streamIdentifier) + .collect(Collectors.toSet()), scheduler.staleStreamDeletionMap() + .keySet()); + assertEquals(0, syncedStreams1.size()); + + StreamConfig deletedStreamConfig = createDummyStreamConfig(5); + // One stream is deleted from Kinesis side + scheduler.deletedStreamListProvider().add(deletedStreamConfig.streamIdentifier()); + + Set syncedStreams2 = scheduler.checkAndSyncStreamShardsAndLeases(); + + Set expectedCurrentStreamConfigs = Sets.newHashSet(streamConfigList1); + expectedCurrentStreamConfigs.remove(deletedStreamConfig); + + //assert kinesis deleted stream is cleaned up from KCL in memory state. + assertEquals(expectedCurrentStreamConfigs, Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + assertEquals(Sets.newHashSet(createDummyStreamConfig(4).streamIdentifier()), + Sets.newHashSet(scheduler.staleStreamDeletionMap().keySet())); + assertEquals(1, syncedStreams2.size()); + assertEquals(0, scheduler.deletedStreamListProvider().purgeAllDeletedStream().size()); + + verify(multiStreamTracker, times(3)).streamConfigList(); + + } + + private void prepareForStaleDeletedStreamCleanupTests() { + + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofDays(1); + } + }); + + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + } + // Tests validate that no cleanup of stream is done if its still tracked in multiStreamTracker + @Test + public void testKinesisStaleDeletedStreamNoCleanUpForTrackedStream() + throws ProvisionedThroughputException, InvalidStateException, DependencyException { + List streamConfigList1 = createDummyStreamConfigList(1,6); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1); + prepareForStaleDeletedStreamCleanupTests(); + + scheduler.deletedStreamListProvider().add(createDummyStreamConfig(3).streamIdentifier()); + + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + + assertEquals(0, syncedStreams.size()); + assertEquals(0, scheduler.staleStreamDeletionMap().size()); + assertEquals(Sets.newHashSet(streamConfigList1), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + } + + //Creates list of upperBound-lowerBound no of dummy StreamConfig + private List createDummyStreamConfigList(int lowerBound, int upperBound) { + return IntStream.range(lowerBound, upperBound).mapToObj(this::createDummyStreamConfig) + .collect(Collectors.toCollection(LinkedList::new)); + } + private StreamConfig createDummyStreamConfig(int streamId){ + return new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)); + } + @Test public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreDeletedAfterDefermentPeriod() throws DependencyException, ProvisionedThroughputException, InvalidStateException { @@ -1114,7 +1192,7 @@ public class SchedulerTest { @Override public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, - StreamConfig streamConfig) { + StreamConfig streamConfig, DeletedStreamListProvider deletedStreamListProvider) { if(shouldReturnDefaultShardSyncTaskmanager) { return shardSyncTaskManager; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index c8ef05ba..a0dbd1f5 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -19,6 +19,7 @@ package software.amazon.kinesis.leases; // import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeast; @@ -54,6 +55,7 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.model.HashKeyRange; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; @@ -62,9 +64,12 @@ import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.coordinator.DeletedStreamListProvider; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.NullMetricsScope; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @@ -292,7 +297,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); @@ -315,7 +320,8 @@ public class HierarchicalShardSyncerTest { extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - verify(shardDetector).listShards(); + verify(shardDetector, never()).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -326,7 +332,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); setupMultiStream(); @@ -349,7 +355,8 @@ public class HierarchicalShardSyncerTest { extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - verify(shardDetector).listShards(); + verify(shardDetector, never()).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -361,7 +368,7 @@ public class HierarchicalShardSyncerTest { /** * Test checkAndCreateLeaseForNewShards with a pre-fetched list of shards. In this scenario, shardDetector.listShards() - * should never be called. + * or shardDetector.listShardsWithoutConsumingResourceNotFoundException() should never be called. */ @Test public void testCheckAndCreateLeasesForShardsWithShardList() throws Exception { @@ -394,13 +401,14 @@ public class HierarchicalShardSyncerTest { extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); verify(shardDetector, never()).listShards(); + verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } /** * Test checkAndCreateLeaseForNewShards with a pre-fetched list of shards. In this scenario, shardDetector.listShards() - * should never be called. + * or shardDetector.listShardsWithoutConsumingResourceNotFoundException() should never be called. */ @Test public void testCheckAndCreateLeasesForShardsWithShardListMultiStream() throws Exception { @@ -431,13 +439,14 @@ public class HierarchicalShardSyncerTest { extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); verify(shardDetector, never()).listShards(); + verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } /** * Test checkAndCreateLeaseForNewShards with an empty list of shards. In this scenario, shardDetector.listShards() - * should never be called. + * or shardDetector.listShardsWithoutConsumingResourceNotFoundException() should never be called. */ @Test public void testCheckAndCreateLeasesForShardsWithEmptyShardList() throws Exception { @@ -463,6 +472,7 @@ public class HierarchicalShardSyncerTest { assertThat(extendedSequenceNumbers.size(), equalTo(0)); verify(shardDetector, never()).listShards(); + verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -668,13 +678,13 @@ public class HierarchicalShardSyncerTest { shards.remove(3); shards.add(3, shard); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { - verify(shardDetector).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, never()).listLeases(); } } @@ -688,14 +698,14 @@ public class HierarchicalShardSyncerTest { shards.remove(3); shards.add(3, shard); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); setupMultiStream(); try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { - verify(shardDetector).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, never()).listLeases(); } } @@ -722,7 +732,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); @@ -743,7 +753,8 @@ public class HierarchicalShardSyncerTest { leaseSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - verify(shardDetector).listShards(); + verify(shardDetector, never()).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -767,7 +778,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); setupMultiStream(); @@ -788,7 +799,7 @@ public class HierarchicalShardSyncerTest { leaseSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - verify(shardDetector).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -822,7 +833,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()).thenReturn(leases); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())).thenReturn(true); doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); @@ -837,7 +848,7 @@ public class HierarchicalShardSyncerTest { assertThat(createLeases, equalTo(expectedCreateLeases)); - verify(shardDetector, times(1)).listShards(); + verify(shardDetector, times(1)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); @@ -849,7 +860,7 @@ public class HierarchicalShardSyncerTest { assertThat(deleteLeases.size(), equalTo(0)); - verify(shardDetector, times(2)).listShards(); + verify(shardDetector, times(2)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(2)).listLeases(); } @@ -885,7 +896,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()) .thenThrow(new DependencyException(new Throwable("Throw for ListLeases"))) .thenReturn(Collections.emptyList()).thenReturn(leases); @@ -897,7 +908,7 @@ public class HierarchicalShardSyncerTest { .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { - verify(shardDetector, times(1)).listShards(); + verify(shardDetector, times(1)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); @@ -912,7 +923,7 @@ public class HierarchicalShardSyncerTest { assertThat(createLeases, equalTo(expectedCreateLeases)); - verify(shardDetector, times(2)).listShards(); + verify(shardDetector, times(2)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(2)).listLeases(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); @@ -927,13 +938,36 @@ public class HierarchicalShardSyncerTest { final Set expectedSequenceNumbers = new HashSet<>( Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - verify(shardDetector, times(3)).listShards(); + verify(shardDetector, times(3)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } } + @Test + public void testDeletedStreamListProviderUpdateOnResourceNotFound() + throws ProvisionedThroughputException, InvalidStateException, DependencyException, InterruptedException { + DeletedStreamListProvider dummyDeletedStreamListProvider = new DeletedStreamListProvider(); + hierarchicalShardSyncer = new HierarchicalShardSyncer(MULTISTREAM_MODE_ON, STREAM_IDENTIFIER, + dummyDeletedStreamListProvider); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenThrow( + ResourceNotFoundException.builder() + .build()); + boolean response = hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, + INITIAL_POSITION_TRIM_HORIZON, SCOPE, ignoreUnexpectedChildShards, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + Set deletedStreamSet = dummyDeletedStreamListProvider.purgeAllDeletedStream(); + + assertFalse(response); + assertThat(deletedStreamSet.size(), equalTo(1)); + assertThat(deletedStreamSet.iterator().next().toString(), equalTo(STREAM_IDENTIFIER)); + + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); + verify(shardDetector, never()).listShards(); + } + @Test(expected = DependencyException.class) public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithCreateLeaseExceptions() throws Exception { @@ -965,7 +999,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()) .thenReturn(Collections.emptyList()).thenReturn(leases); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())) @@ -977,7 +1011,7 @@ public class HierarchicalShardSyncerTest { .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { - verify(shardDetector, times(1)).listShards(); + verify(shardDetector, times(1)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); verify(dynamoDBLeaseRefresher, times(1)).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); @@ -991,7 +1025,7 @@ public class HierarchicalShardSyncerTest { final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); - verify(shardDetector, times(2)).listShards(); + verify(shardDetector, times(2)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(2)).listLeases(); verify(dynamoDBLeaseRefresher, times(1 + expectedCreateLeases.size())) .createLeaseIfNotExists(any(Lease.class)); @@ -1002,7 +1036,7 @@ public class HierarchicalShardSyncerTest { .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - verify(shardDetector, times(3)).listShards(); + verify(shardDetector, times(3)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(1 + expectedCreateLeases.size())) .createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); @@ -1084,7 +1118,7 @@ public class HierarchicalShardSyncerTest { final List existingLeases) throws Exception { final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); when(shardDetector.listShardsWithFilter(any())).thenReturn(getFilteredShards(shards, initialPosition)); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(existingLeases); when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(existingLeases.isEmpty()); @@ -2381,14 +2415,14 @@ public class HierarchicalShardSyncerTest { final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); - when(shardDetector.listShards()).thenReturn(shardsWithoutLeases); + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shardsWithoutLeases); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(currentLeases); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - verify(shardDetector, atLeast(1)).listShards(); + verify(shardDetector, atLeast(1)).listShardsWithoutConsumingResourceNotFoundException(); } /** From 504ea10859413b2738ca911288ffc45d0af8c5eb Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Fri, 10 Mar 2023 15:07:46 -0500 Subject: [PATCH 29/88] Fixed NPE in `LeaseCleanupManager`. (#1061) --- .../kinesis/leases/LeaseCleanupManager.java | 4 +- .../leases/LeaseCleanupManagerTest.java | 102 ++++++++---------- 2 files changed, 48 insertions(+), 58 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java index f9e52e1c..d62cd476 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java @@ -117,8 +117,8 @@ public class LeaseCleanupManager { public void enqueueForDeletion(LeasePendingDeletion leasePendingDeletion) { final Lease lease = leasePendingDeletion.lease(); if (lease == null) { - log.warn("Cannot enqueue lease {} for deferred deletion - instance doesn't hold the lease for that shard.", - lease.leaseKey()); + log.warn("Cannot enqueue {} for {} as instance doesn't hold the lease for that shard.", + leasePendingDeletion.shardInfo(), leasePendingDeletion.streamIdentifier()); } else { log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey()); if (!deletionQueue.add(leasePendingDeletion)) { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java index 2e691844..9a731f80 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java @@ -45,19 +45,14 @@ import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) public class LeaseCleanupManagerTest { - private ShardInfo shardInfo; - private StreamIdentifier streamIdentifier; - private String concurrencyToken = "1234"; + private static final ShardInfo SHARD_INFO = new ShardInfo("shardId", "concurrencyToken", + Collections.emptySet(), ExtendedSequenceNumber.LATEST); - private String shardId = "shardId"; - private String splitParent = "splitParent"; - private String mergeParent1 = "mergeParent-1"; - private String mergeParent2 = "mergeParent-2"; + private static final StreamIdentifier STREAM_IDENTIFIER = StreamIdentifier.singleStreamInstance("streamName"); - private Duration maxFutureWait = Duration.ofSeconds(1); - private long leaseCleanupIntervalMillis = Duration.ofSeconds(1).toMillis(); - private long completedLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); - private long garbageLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); + private final long leaseCleanupIntervalMillis = Duration.ofSeconds(1).toMillis(); + private final long completedLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); + private final long garbageLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); private boolean cleanupLeasesOfCompletedShards = true; private LeaseCleanupManager leaseCleanupManager; private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory(); @@ -73,9 +68,6 @@ public class LeaseCleanupManagerTest { @Before public void setUp() throws Exception { - shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - streamIdentifier = StreamIdentifier.singleStreamInstance("streamName"); leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); @@ -112,10 +104,8 @@ public class LeaseCleanupManagerTest { */ @Test public final void testParentShardLeaseDeletedSplitCase() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - - verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 1); + verifyExpectedDeletedLeasesCompletedShardCase(SHARD_INFO, childShardsForSplit(), + ExtendedSequenceNumber.LATEST, 1); } /** @@ -124,10 +114,8 @@ public class LeaseCleanupManagerTest { */ @Test public final void testParentShardLeaseDeletedMergeCase() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - - verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 1); + verifyExpectedDeletedLeasesCompletedShardCase(SHARD_INFO, childShardsForMerge(), + ExtendedSequenceNumber.LATEST, 1); } /** @@ -136,15 +124,14 @@ public class LeaseCleanupManagerTest { */ @Test public final void testNoLeasesDeletedWhenNotEnabled() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); cleanupLeasesOfCompletedShards = false; leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); - verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 0); + verifyExpectedDeletedLeasesCompletedShardCase(SHARD_INFO, childShardsForSplit(), + ExtendedSequenceNumber.LATEST, 0); } /** @@ -155,10 +142,8 @@ public class LeaseCleanupManagerTest { public final void testNoCleanupWhenSomeChildShardLeasesAreNotPresent() throws Exception { List childShards = childShardsForSplit(); - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - - verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, ExtendedSequenceNumber.LATEST, false, 0); + verifyExpectedDeletedLeasesCompletedShardCase(SHARD_INFO, childShards, + ExtendedSequenceNumber.LATEST, false, 0); } /** @@ -179,12 +164,9 @@ public class LeaseCleanupManagerTest { testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.AT_TIMESTAMP); } - private final void testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber extendedSequenceNumber) + private void testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber extendedSequenceNumber) throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - - verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), extendedSequenceNumber, 0); + verifyExpectedDeletedLeasesCompletedShardCase(SHARD_INFO, childShardsForMerge(), extendedSequenceNumber, 0); } /** @@ -192,33 +174,38 @@ public class LeaseCleanupManagerTest { */ @Test public final void testLeaseNotDeletedWhenParentsStillPresent() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.singleton("parent"), + final ShardInfo shardInfo = new ShardInfo("shardId-0", "concurrencyToken", Collections.singleton("parent"), ExtendedSequenceNumber.LATEST); verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 0); } + /** + * Verify {@link NullPointerException} is not thrown when a null lease is enqueued. + */ + @Test + public void testEnqueueNullLease() { + leaseCleanupManager.enqueueForDeletion(createLeasePendingDeletion(null, SHARD_INFO)); + } + /** * Tests ResourceNotFound case for if a shard expires, that we delete the lease when shardExpired is found. */ @Test public final void testLeaseDeletedWhenShardDoesNotExist() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + final Lease heldLease = LeaseHelper.createLease(SHARD_INFO.shardId(), "leaseOwner", + Collections.singleton("parentShardId")); testLeaseDeletedWhenShardDoesNotExist(heldLease); } /** * Tests ResourceNotFound case when completed lease cleanup is disabled. - * @throws Exception */ @Test public final void testLeaseDeletedWhenShardDoesNotExistAndCleanupCompletedLeaseDisabled() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + final Lease heldLease = LeaseHelper.createLease(SHARD_INFO.shardId(), "leaseOwner", + Collections.singleton("parentShardId")); cleanupLeasesOfCompletedShards = false; @@ -229,32 +216,31 @@ public class LeaseCleanupManagerTest { testLeaseDeletedWhenShardDoesNotExist(heldLease); } - public void testLeaseDeletedWhenShardDoesNotExist(Lease heldLease) throws Exception { + private void testLeaseDeletedWhenShardDoesNotExist(Lease heldLease) throws Exception { when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); + when(leaseCoordinator.getCurrentlyHeldLease(SHARD_INFO.shardId())).thenReturn(heldLease); when(shardDetector.getChildShards(any(String.class))).thenThrow(ResourceNotFoundException.class); when(leaseRefresher.getLease(heldLease.leaseKey())).thenReturn(heldLease); - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo, shardDetector)); + leaseCleanupManager.enqueueForDeletion(createLeasePendingDeletion(heldLease, SHARD_INFO)); leaseCleanupManager.cleanupLeases(); - verify(shardDetector, times(1)).getChildShards(shardInfo.shardId()); - verify(leaseRefresher, times(1)).deleteLease(heldLease); + verify(shardDetector).getChildShards(SHARD_INFO.shardId()); + verify(leaseRefresher).deleteLease(heldLease); } - private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + private void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, ExtendedSequenceNumber extendedSequenceNumber, int expectedDeletedLeases) throws Exception { verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, extendedSequenceNumber, true, expectedDeletedLeases); } - private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + private void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, ExtendedSequenceNumber extendedSequenceNumber, boolean childShardLeasesPresent, int expectedDeletedLeases) throws Exception { - final Lease lease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", shardInfo.parentShardIds(), - childShards.stream().map(c -> c.shardId()).collect(Collectors.toSet())); + childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet())); final List childShardLeases = childShards.stream().map(c -> LeaseHelper.createLease( ShardInfo.getLeaseKey(shardInfo, c.shardId()), "leaseOwner", Collections.singleton(shardInfo.shardId()), Collections.emptyList(), extendedSequenceNumber)).collect(Collectors.toList()); @@ -273,15 +259,15 @@ public class LeaseCleanupManagerTest { } } - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, lease, shardInfo, shardDetector)); + leaseCleanupManager.enqueueForDeletion(createLeasePendingDeletion(lease, shardInfo)); leaseCleanupManager.cleanupLeases(); - verify(shardDetector, times(1)).getChildShards(shardInfo.shardId()); + verify(shardDetector).getChildShards(shardInfo.shardId()); verify(leaseRefresher, times(expectedDeletedLeases)).deleteLease(any(Lease.class)); } private List childShardsForSplit() { - List parentShards = Arrays.asList(splitParent); + final List parentShards = Collections.singletonList("splitParent"); ChildShard leftChild = ChildShard.builder() .shardId("leftChild") @@ -294,11 +280,11 @@ public class LeaseCleanupManagerTest { .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) .build(); - return Arrays.asList(leftChild, rightChild); + return Arrays.asList(leftChild, rightChild); } private List childShardsForMerge() { - List parentShards = Arrays.asList(mergeParent1, mergeParent2); + final List parentShards = Arrays.asList("mergeParent1", "mergeParent2"); ChildShard child = ChildShard.builder() .shardId("onlyChild") @@ -308,4 +294,8 @@ public class LeaseCleanupManagerTest { return Collections.singletonList(child); } + + private LeasePendingDeletion createLeasePendingDeletion(final Lease lease, final ShardInfo shardInfo) { + return new LeasePendingDeletion(STREAM_IDENTIFIER, lease, shardInfo, shardDetector); + } } From c4204002af350465c6633e96d5c7bf1950f89bf6 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 13 Mar 2023 19:57:01 -0400 Subject: [PATCH 30/88] Fixed retry storm in `PrefetchRecordsPublisher`. (#1062) + DRY in `PrefetchRecordsPublisherTest` --- .../polling/PrefetchRecordsPublisher.java | 3 + .../polling/PrefetchRecordsPublisherTest.java | 157 +++++++++--------- 2 files changed, 77 insertions(+), 83 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index ab406244..07f4aaac 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -563,6 +563,9 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { if (timeSinceLastCall < idleMillisBetweenCalls) { Thread.sleep(idleMillisBetweenCalls - timeSinceLastCall); } + + // avoid immediate-retry storms + lastSuccessfulCall = null; } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java index 55d76432..74707eb4 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -32,13 +31,12 @@ import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static software.amazon.kinesis.utils.BlockingUtils.blockUntilConditionSatisfied; -import static software.amazon.kinesis.utils.BlockingUtils.blockUntilRecordsAvailable; import static software.amazon.kinesis.utils.ProcessRecordsInputMatcher.eqProcessRecordsInput; import java.time.Duration; @@ -83,7 +81,6 @@ import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.kinesis.common.InitialPositionInStreamExtended; -import software.amazon.kinesis.common.RequestDetails; import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.lifecycle.ShardConsumerNotifyingSubscriber; @@ -95,6 +92,7 @@ import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RecordsRetrieved; import software.amazon.kinesis.retrieval.RetryableRetrievalException; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import software.amazon.kinesis.utils.BlockingUtils; /** * Test class for the PrefetchRecordsPublisher class. @@ -107,10 +105,10 @@ public class PrefetchRecordsPublisherTest { private static final int MAX_RECORDS_PER_CALL = 10000; private static final int MAX_SIZE = 5; private static final int MAX_RECORDS_COUNT = 15000; - private static final long IDLE_MILLIS_BETWEEN_CALLS = 0L; - private static final long AWAIT_TERMINATION_TIMEOUT = 1L; private static final String NEXT_SHARD_ITERATOR = "testNextShardIterator"; + private static final long DEFAULT_TIMEOUT_MILLIS = Duration.ofSeconds(1).toMillis(); + @Mock private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @Mock @@ -124,28 +122,15 @@ public class PrefetchRecordsPublisherTest { private ExecutorService executorService; private LinkedBlockingQueue spyQueue; private PrefetchRecordsPublisher getRecordsCache; - private String operation = "ProcessTask"; private GetRecordsResponse getRecordsResponse; private Record record; - private RequestDetails requestDetails; @Before public void setup() { when(getRecordsRetrievalStrategy.dataFetcher()).thenReturn(dataFetcher); when(dataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("testStream")); executorService = spy(Executors.newFixedThreadPool(1)); - getRecordsCache = new PrefetchRecordsPublisher( - MAX_SIZE, - 3 * SIZE_1_MB, - MAX_RECORDS_COUNT, - MAX_RECORDS_PER_CALL, - getRecordsRetrievalStrategy, - executorService, - IDLE_MILLIS_BETWEEN_CALLS, - new NullMetricsFactory(), - operation, - "shardId", - AWAIT_TERMINATION_TIMEOUT); + getRecordsCache = createPrefetchRecordsPublisher(0L); spyQueue = spy(getRecordsCache.getPublisherSession().prefetchRecordsQueue()); records = spy(new ArrayList<>()); getRecordsResponse = GetRecordsResponse.builder().records(records).nextShardIterator(NEXT_SHARD_ITERATOR).childShards(new ArrayList<>()).build(); @@ -158,7 +143,7 @@ public class PrefetchRecordsPublisherTest { getRecordsCache.start(sequenceNumber, initialPosition); getRecordsCache.start(sequenceNumber, initialPosition); getRecordsCache.start(sequenceNumber, initialPosition); - verify(dataFetcher, times(1)).initialize(any(ExtendedSequenceNumber.class), any()); + verify(dataFetcher).initialize(any(ExtendedSequenceNumber.class), any()); } @Test @@ -189,7 +174,7 @@ public class PrefetchRecordsPublisherTest { } private void verifyInternalState(int queueSize) { - Assert.assertTrue(getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == queueSize); + assertEquals(queueSize, getRecordsCache.getPublisherSession().prefetchRecordsQueue().size()); } @Test @@ -202,9 +187,7 @@ public class PrefetchRecordsPublisherTest { .map(KinesisClientRecord::fromRecord).collect(Collectors.toList()); getRecordsCache.start(sequenceNumber, initialPosition); - ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, - "shardId"), 1000L) - .processRecordsInput(); + ProcessRecordsInput result = blockUntilRecordsAvailable().processRecordsInput(); assertEquals(expectedRecords, result.records()); assertEquals(new ArrayList<>(), result.childShards()); @@ -215,19 +198,7 @@ public class PrefetchRecordsPublisherTest { @Test(expected = RuntimeException.class) public void testGetRecordsWithInitialFailures_LessThanRequiredWait_Throws() { - // Create a new PrefetchRecordsPublisher with 1s idle time between get calls - getRecordsCache = new PrefetchRecordsPublisher( - MAX_SIZE, - 3 * SIZE_1_MB, - MAX_RECORDS_COUNT, - MAX_RECORDS_PER_CALL, - getRecordsRetrievalStrategy, - executorService, - 1000, - new NullMetricsFactory(), - operation, - "shardId", - AWAIT_TERMINATION_TIMEOUT); + getRecordsCache = createPrefetchRecordsPublisher(Duration.ofSeconds(1).toMillis()); // Setup the retrieval strategy to fail initial calls before succeeding when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenThrow(new RetryableRetrievalException("Timed out")).thenThrow(new @@ -236,33 +207,15 @@ public class PrefetchRecordsPublisherTest { when(records.size()).thenReturn(1000); - final List expectedRecords = records.stream() - .map(KinesisClientRecord::fromRecord).collect(Collectors.toList()); - getRecordsCache.start(sequenceNumber, initialPosition); - ProcessRecordsInput result = null; // Setup timeout to be less than what the PrefetchRecordsPublisher will need based on the idle time between // get calls to validate exception is thrown - result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, - "shardId"), 1000L) - .processRecordsInput(); + blockUntilRecordsAvailable(); } @Test public void testGetRecordsWithInitialFailures_AdequateWait_Success() { - // Create a new PrefetchRecordsPublisher with 1s idle time between get calls - getRecordsCache = new PrefetchRecordsPublisher( - MAX_SIZE, - 3 * SIZE_1_MB, - MAX_RECORDS_COUNT, - MAX_RECORDS_PER_CALL, - getRecordsRetrievalStrategy, - executorService, - 1000, - new NullMetricsFactory(), - operation, - "shardId", - AWAIT_TERMINATION_TIMEOUT); + getRecordsCache = createPrefetchRecordsPublisher(Duration.ofSeconds(1).toMillis()); // Setup the retrieval strategy to fail initial calls before succeeding when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenThrow(new RetryableRetrievalException("Timed out")).thenThrow(new @@ -278,8 +231,7 @@ public class PrefetchRecordsPublisherTest { ProcessRecordsInput result = null; // Setup timeout to be more than what the PrefetchRecordsPublisher will need based on the idle time between // get calls and then validate the mocks later - result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, - "shardId"), 4000L) + result = BlockingUtils.blockUntilRecordsAvailable(this::evictPublishedEvent, 4000L) .processRecordsInput(); assertEquals(expectedRecords, result.records()); @@ -303,8 +255,7 @@ public class PrefetchRecordsPublisherTest { getRecordsCache.start(sequenceNumber, initialPosition); try { - ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L) - .processRecordsInput(); + blockUntilRecordsAvailable(); } catch (Exception e) { assertEquals("No records found", e.getMessage()); } @@ -337,8 +288,7 @@ public class PrefetchRecordsPublisherTest { when(dataFetcher.isShardEndReached()).thenReturn(true); getRecordsCache.start(sequenceNumber, initialPosition); - ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L) - .processRecordsInput(); + ProcessRecordsInput result = blockUntilRecordsAvailable().processRecordsInput(); assertEquals(expectedRecords, result.records()); assertEquals(childShards, result.childShards()); @@ -406,7 +356,7 @@ public class PrefetchRecordsPublisherTest { .map(KinesisClientRecord::fromRecord).collect(Collectors.toList()); getRecordsCache.start(sequenceNumber, initialPosition); - ProcessRecordsInput processRecordsInput = evictPublishedEvent(getRecordsCache, "shardId").processRecordsInput(); + ProcessRecordsInput processRecordsInput = evictPublishedEvent().processRecordsInput(); verify(executorService).execute(any()); assertEquals(expectedRecords, processRecordsInput.records()); @@ -415,7 +365,7 @@ public class PrefetchRecordsPublisherTest { sleep(2000); - ProcessRecordsInput processRecordsInput2 = evictPublishedEvent(getRecordsCache, "shardId").processRecordsInput(); + ProcessRecordsInput processRecordsInput2 = evictPublishedEvent().processRecordsInput(); assertNotEquals(processRecordsInput, processRecordsInput2); assertEquals(expectedRecords, processRecordsInput2.records()); assertNotEquals(processRecordsInput2.timeSpentInCache(), Duration.ZERO); @@ -425,7 +375,7 @@ public class PrefetchRecordsPublisherTest { @Test(expected = IllegalStateException.class) public void testGetNextRecordsWithoutStarting() { - verify(executorService, times(0)).execute(any()); + verify(executorService, never()).execute(any()); getRecordsCache.drainQueueForRequests(); } @@ -437,7 +387,6 @@ public class PrefetchRecordsPublisherTest { @Test public void testExpiredIteratorException() { - log.info("Starting tests"); when(getRecordsRetrievalStrategy.getRecords(MAX_RECORDS_PER_CALL)).thenThrow(ExpiredIteratorException.class) .thenReturn(getRecordsResponse); @@ -445,7 +394,7 @@ public class PrefetchRecordsPublisherTest { doNothing().when(dataFetcher).restartIterator(); - blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L); + blockUntilRecordsAvailable(); sleep(1000); @@ -456,7 +405,6 @@ public class PrefetchRecordsPublisherTest { public void testExpiredIteratorExceptionWithIllegalStateException() { // This test validates that the daemon thread doesn't die when ExpiredIteratorException occurs with an // IllegalStateException. - when(getRecordsRetrievalStrategy.getRecords(MAX_RECORDS_PER_CALL)) .thenThrow(ExpiredIteratorException.builder().build()) .thenReturn(getRecordsResponse) @@ -474,14 +422,13 @@ public class PrefetchRecordsPublisherTest { @Test public void testRetryableRetrievalExceptionContinues() { - GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L).records(Collections.emptyList()).nextShardIterator(NEXT_SHARD_ITERATOR).build(); when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenThrow(new RetryableRetrievalException("Timeout", new TimeoutException("Timeout"))).thenReturn(response); getRecordsCache.start(sequenceNumber, initialPosition); - RecordsRetrieved records = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000); - assertThat(records.processRecordsInput().millisBehindLatest(), equalTo(response.millisBehindLatest())); + RecordsRetrieved records = blockUntilRecordsAvailable(); + assertEquals(records.processRecordsInput().millisBehindLatest(), response.millisBehindLatest()); } @Test(timeout = 10000L) @@ -493,7 +440,6 @@ public class PrefetchRecordsPublisherTest { // If the test times out before starting the subscriber it means something went wrong while filling the queue. // After the subscriber is started one of the things that can trigger a timeout is a deadlock. // - final int[] sequenceNumberInResponse = { 0 }; when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenAnswer( i -> GetRecordsResponse.builder().records( @@ -681,14 +627,14 @@ public class PrefetchRecordsPublisherTest { getRecordsCache.start(sequenceNumber, initialPosition); - RecordsRetrieved lastProcessed = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000); - RecordsRetrieved expected = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000); + RecordsRetrieved lastProcessed = blockUntilRecordsAvailable(); + RecordsRetrieved expected = blockUntilRecordsAvailable(); // // Skip some of the records the cache // - blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000); - blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000); + blockUntilRecordsAvailable(); + blockUntilRecordsAvailable(); verify(getRecordsRetrievalStrategy, atLeast(2)).getRecords(anyInt()); @@ -697,16 +643,46 @@ public class PrefetchRecordsPublisherTest { } getRecordsCache.restartFrom(lastProcessed); - RecordsRetrieved postRestart = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000); + RecordsRetrieved postRestart = blockUntilRecordsAvailable(); assertThat(postRestart.processRecordsInput(), eqProcessRecordsInput(expected.processRecordsInput())); verify(dataFetcher).resetIterator(eq(responses.get(0).nextShardIterator()), eq(responses.get(0).records().get(0).sequenceNumber()), any()); - } - private RecordsRetrieved evictPublishedEvent(PrefetchRecordsPublisher publisher, String shardId) { - return publisher.getPublisherSession().evictPublishedRecordAndUpdateDemand(shardId); + /** + * Tests that a thrown {@link SdkException} doesn't cause a retry storm. + */ + @Test(expected = RuntimeException.class) + public void testRepeatSdkExceptionLoop() { + final int expectedFailedCalls = 4; + getRecordsCache = createPrefetchRecordsPublisher(DEFAULT_TIMEOUT_MILLIS / expectedFailedCalls); + getRecordsCache.start(sequenceNumber, initialPosition); + + try { + // return a valid response to cause `lastSuccessfulCall` to initialize + when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenReturn(GetRecordsResponse.builder().build()); + blockUntilRecordsAvailable(); + } catch (RuntimeException re) { + Assert.fail("first call should succeed"); + } + + try { + when(getRecordsRetrievalStrategy.getRecords(anyInt())) + .thenThrow(SdkException.builder().message("lose yourself to dance").build()); + blockUntilRecordsAvailable(); + } finally { + // the successful call is the +1 + verify(getRecordsRetrievalStrategy, times(expectedFailedCalls + 1)).getRecords(anyInt()); + } + } + + private RecordsRetrieved blockUntilRecordsAvailable() { + return BlockingUtils.blockUntilRecordsAvailable(this::evictPublishedEvent, DEFAULT_TIMEOUT_MILLIS); + } + + private RecordsRetrieved evictPublishedEvent() { + return getRecordsCache.getPublisherSession().evictPublishedRecordAndUpdateDemand("shardId"); } private static class RetrieverAnswer implements Answer { @@ -736,7 +712,7 @@ public class PrefetchRecordsPublisherTest { } @Override - public GetRecordsResponse answer(InvocationOnMock invocation) throws Throwable { + public GetRecordsResponse answer(InvocationOnMock invocation) { GetRecordsResponse response = iterator.next(); if (!iterator.hasNext()) { iterator = responses.iterator(); @@ -787,4 +763,19 @@ public class PrefetchRecordsPublisherTest { return SdkBytes.fromByteArray(new byte[size]); } + private PrefetchRecordsPublisher createPrefetchRecordsPublisher(final long idleMillisBetweenCalls) { + return new PrefetchRecordsPublisher( + MAX_SIZE, + 3 * SIZE_1_MB, + MAX_RECORDS_COUNT, + MAX_RECORDS_PER_CALL, + getRecordsRetrievalStrategy, + executorService, + idleMillisBetweenCalls, + new NullMetricsFactory(), + PrefetchRecordsPublisherTest.class.getSimpleName(), + "shardId", + 1L); + } + } From 5bbb9768b5c1d702b526ff260da1e2a33c3add9f Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 13 Mar 2023 20:05:40 -0400 Subject: [PATCH 31/88] DRY: simplification of `HierarchicalShardSyncerTest`. (#1059) --- .../leases/HierarchicalShardSyncer.java | 5 - .../leases/HierarchicalShardSyncerTest.java | 682 +++++++----------- 2 files changed, 274 insertions(+), 413 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index aafbfcff..e191e5d0 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -134,9 +134,6 @@ public class HierarchicalShardSyncer { final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, List latestShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - - //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 - if (!CollectionUtils.isNullOrEmpty(latestShards)) { log.debug("{} - Num shards: {}", streamIdentifier, latestShards.size()); } else { @@ -176,8 +173,6 @@ public class HierarchicalShardSyncer { } } log.info("{} - Newly created leases {}: {}", streamIdentifier, createdLeases.size(), createdLeases); - final List trackedLeases = new ArrayList<>(currentLeases); - trackedLeases.addAll(newLeasesToCreate); return true; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index a0dbd1f5..d9f36481 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -19,8 +19,10 @@ package software.amazon.kinesis.leases; // import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doNothing; @@ -47,6 +49,7 @@ import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.Validate; +import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -75,6 +78,7 @@ import software.amazon.kinesis.metrics.NullMetricsScope; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import static software.amazon.kinesis.leases.HierarchicalShardSyncer.MemoizationContext; +import static software.amazon.kinesis.leases.HierarchicalShardSyncer.determineNewLeasesToCreate; @RunWith(MockitoJUnitRunner.class) public class HierarchicalShardSyncerTest { @@ -85,13 +89,51 @@ public class HierarchicalShardSyncerTest { private static final InitialPositionInStreamExtended INITIAL_POSITION_AT_TIMESTAMP = InitialPositionInStreamExtended .newInitialPositionAtTimestamp(new Date(1000L)); private static final int EXPONENT = 128; - private static final String LEASE_OWNER = "TestOwnere"; + private static final String LEASE_OWNER = "TestOwner"; private static final MetricsScope SCOPE = new NullMetricsScope(); private static final boolean MULTISTREAM_MODE_ON = true; private static final String STREAM_IDENTIFIER = "acc:stream:1"; private static final HierarchicalShardSyncer.MultiStreamArgs MULTI_STREAM_ARGS = new HierarchicalShardSyncer.MultiStreamArgs( MULTISTREAM_MODE_ON, StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); + /** + *
+     * Shard structure (y-axis is
+     * epochs): 0 1 2 3 4   5- shards till
+     *          \ / \ / |   |
+     *           6   7  4   5- shards from epoch 103 - 205
+     *            \ /   |  /\
+     *             8    4 9 10 -
+     * shards from epoch 206 (open - no ending sequenceNumber)
+     * 
+ */ + private static final List SHARD_GRAPH_A = Collections.unmodifiableList(constructShardListForGraphA()); + + /** + * Shard structure (x-axis is epochs): + *
+     * 0  3   6   9
+     * \ / \ / \ /
+     *  2   5   8
+     * / \ / \ / \
+     * 1  4   7  10
+     * 
+ */ + private static final List SHARD_GRAPH_B = Collections.unmodifiableList(constructShardListForGraphB()); + + /** + *
+     * Shard structure (y-axis is
+     * epochs):     0      1  2  3  - shards till
+     *            /   \    |  \ /
+     *           4     5   1   6  - shards from epoch 103 - 205
+     *          / \   / \  |   |
+     *         7   8 9  10 1   6
+     * shards from epoch 206 (open - no ending sequenceNumber)
+     * 
+ */ + private static final List SHARD_GRAPH_C = Collections.unmodifiableList(constructShardListForGraphC()); + private final boolean ignoreUnexpectedChildShards = false; private HierarchicalShardSyncer hierarchicalShardSyncer; @@ -125,8 +167,8 @@ public class HierarchicalShardSyncerTest { final List shards = Collections.emptyList(); final List leases = Collections.emptyList(); final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - assertThat(HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, - INITIAL_POSITION_LATEST).isEmpty(), equalTo(true)); + assertTrue(determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, INITIAL_POSITION_LATEST) + .isEmpty()); } /** @@ -137,9 +179,8 @@ public class HierarchicalShardSyncerTest { final List leases = Collections.emptyList(); final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - assertThat(HierarchicalShardSyncer - .determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, INITIAL_POSITION_LATEST, - new HashSet<>(), MULTI_STREAM_ARGS).isEmpty(), equalTo(true)); + assertTrue(determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, INITIAL_POSITION_LATEST, + Collections.emptySet(), MULTI_STREAM_ARGS).isEmpty()); } /** @@ -156,15 +197,9 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Collections.emptyList(); final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, + final List newLeases = determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); - validateHashRangeinLease(newLeases); - - final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); - - assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); - assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + validateLeases(newLeases, shardId0, shardId1); } /** @@ -181,15 +216,9 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Collections.emptyList(); final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, + final List newLeases = determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS); - validateHashRangeinLease(newLeases); - final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set expectedLeaseIds = new HashSet<>( - toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); - - assertThat(newLeases.size(), equalTo(expectedLeaseIds.size())); - assertThat(newLeaseKeys, equalTo(expectedLeaseIds)); + validateLeases(newLeases, toMultiStreamLeases(shardId0, shardId1)); } /** @@ -218,13 +247,9 @@ public class HierarchicalShardSyncerTest { final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, + final List newLeases = determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); - validateHashRangeinLease(newLeases); - final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); - assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); - assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + validateLeases(newLeases, shardId0, shardId1); } /** @@ -245,7 +270,8 @@ public class HierarchicalShardSyncerTest { ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); - final List currentLeases = new ArrayList(createMultiStreamLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo")); + final List currentLeases = new ArrayList<>(createMultiStreamLeasesFromShards(shardsWithLeases, + ExtendedSequenceNumber.LATEST, "foo")); final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); @@ -253,23 +279,29 @@ public class HierarchicalShardSyncerTest { final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, + final List newLeases = determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); - final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - validateHashRangeinLease(newLeases); - final Set expectedLeaseShardIds = new HashSet<>( - toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); - assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); - assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + validateLeases(newLeases, toMultiStreamLeases(shardId0, shardId1)); } - private void validateHashRangeinLease(List leases) { + private static void validateHashRangeInLease(List leases) { final Consumer leaseValidation = lease -> { Validate.notNull(lease.hashKeyRangeForLease()); Validate.isTrue(lease.hashKeyRangeForLease().startingHashKey() .compareTo(lease.hashKeyRangeForLease().endingHashKey()) < 0); }; - leases.forEach(lease -> leaseValidation.accept(lease)); + leases.forEach(leaseValidation); + } + + /** + * Validates that a {@link Lease} exists for each expected lease key. + */ + private static void validateLeases(final List leases, final String... expectedLeaseKeys) { + validateHashRangeInLease(leases); + assertEquals(expectedLeaseKeys.length, leases.size()); + + final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + assertThat(leaseKeys, Matchers.containsInAnyOrder(expectedLeaseKeys)); } /** @@ -288,82 +320,89 @@ public class HierarchicalShardSyncerTest { testCheckAndCreateLeasesForShardsIfMissing(INITIAL_POSITION_LATEST); } + private void testLeaseCreation( + final List shards, + final boolean ignoreUnexpectedChildShards, + final String... expectedLeaseKeys) + throws Exception { + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + + when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + final List requestLeases = leaseCaptor.getAllValues(); + final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) + .collect(Collectors.toSet()); + + validateLeases(requestLeases, expectedLeaseKeys); + assertEquals(1, extendedSequenceNumbers.size()); + + extendedSequenceNumbers.forEach(seq -> assertEquals(ExtendedSequenceNumber.LATEST, seq)); + + verify(shardDetector, never()).listShards(); + verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); + verify(dynamoDBLeaseRefresher, times(requestLeases.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + } + /** * Test checkAndCreateLeaseForNewShards while not providing a pre-fetched list of shards */ @Test public void testCheckAndCreateLeasesForShardsIfMissingAtLatest() throws Exception { - final List shards = constructShardListForGraphA(); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); - - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final Set expectedShardIds = new HashSet<>( - Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); - - final List requestLeases = leaseCaptor.getAllValues(); - final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - validateHashRangeinLease(requestLeases); - assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); - assertThat(requestLeaseKeys, equalTo(expectedShardIds)); - assertThat(extendedSequenceNumbers.size(), equalTo(1)); - - extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - - verify(shardDetector, never()).listShards(); - verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); - verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + testLeaseCreation(SHARD_GRAPH_A, false, + "shardId-4", "shardId-8", "shardId-9", "shardId-10"); } @Test public void testCheckAndCreateLeasesForShardsIfMissingAtLatestMultiStream() throws Exception { - final List shards = constructShardListForGraphA(); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); setupMultiStream(); - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final Set expectedShardIds = new HashSet<>( - toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); - - final List requestLeases = leaseCaptor.getAllValues(); - final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - validateHashRangeinLease(requestLeases); - assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); - assertThat(requestLeaseKeys, equalTo(expectedShardIds)); - assertThat(extendedSequenceNumbers.size(), equalTo(1)); - - extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - - verify(shardDetector, never()).listShards(); - verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); - verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + testLeaseCreation(SHARD_GRAPH_A, false, + toMultiStreamLeases("shardId-4", "shardId-8", "shardId-9", "shardId-10")); } - private List toMultiStreamLeaseList(List shardIdBasedLeases) { - return shardIdBasedLeases.stream().map(s -> STREAM_IDENTIFIER + ":" + s) - .collect(Collectors.toList()); + /** + * Converts one-or-more shard ids to their multi-stream equivalent. + * + * @param shardIds vararg of shard ids (i.e., {@code shardId-}) + * @return a same-sized array where the Nth element is the multi-stream + * equivalent of the Nth {@code shardIds} input + */ + private static String[] toMultiStreamLeases(final String... shardIds) { + final String[] multiStreamLeaseKey = new String[shardIds.length]; + for (int i = 0; i < shardIds.length; i++) { + multiStreamLeaseKey[i] = STREAM_IDENTIFIER + ":" + shardIds[i]; + } + return multiStreamLeaseKey; + } + + private void testCheckAndCreateLeasesForShardsWithShardList(final String... expectedLeaseKeys) throws Exception { + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SHARD_GRAPH_A, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + final List requestLeases = leaseCaptor.getAllValues(); + final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) + .collect(Collectors.toSet()); + + validateLeases(requestLeases, expectedLeaseKeys); + assertEquals(1, extendedSequenceNumbers.size()); + + extendedSequenceNumbers.forEach(seq -> assertEquals(ExtendedSequenceNumber.LATEST, seq)); + + verify(shardDetector, never()).listShards(); + verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); + verify(dynamoDBLeaseRefresher, times(requestLeases.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } /** @@ -372,38 +411,7 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForShardsWithShardList() throws Exception { - final List latestShards = constructShardListForGraphA(); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(latestShards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); - - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - latestShards, false, SCOPE, - dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final Set expectedShardIds = new HashSet<>( - Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); - - final List requestLeases = leaseCaptor.getAllValues(); - final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); - assertThat(requestLeaseKeys, equalTo(expectedShardIds)); - assertThat(extendedSequenceNumbers.size(), equalTo(1)); - - validateHashRangeinLease(requestLeases); - - extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - - verify(shardDetector, never()).listShards(); - verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); - verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + testCheckAndCreateLeasesForShardsWithShardList("shardId-4", "shardId-8", "shardId-9", "shardId-10"); } /** @@ -412,36 +420,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForShardsWithShardListMultiStream() throws Exception { - final List latestShards = constructShardListForGraphA(); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(latestShards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); setupMultiStream(); - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - latestShards, false, SCOPE, - dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final Set expectedShardIds = new HashSet<>( - toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); - - final List requestLeases = leaseCaptor.getAllValues(); - final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); - assertThat(requestLeaseKeys, equalTo(expectedShardIds)); - assertThat(extendedSequenceNumbers.size(), equalTo(1)); - validateHashRangeinLease(requestLeases); - extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - - verify(shardDetector, never()).listShards(); - verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); - verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + testCheckAndCreateLeasesForShardsWithShardList( + toMultiStreamLeases("shardId-4", "shardId-8", "shardId-9", "shardId-10")); } /** @@ -450,26 +431,21 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForShardsWithEmptyShardList() throws Exception { - final List shards = constructShardListForGraphA(); - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - when(shardDetector.listShards()).thenReturn(shards); + when(shardDetector.listShards()).thenReturn(SHARD_GRAPH_A); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - new ArrayList(), false, SCOPE, + new ArrayList<>(), false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - final Set expectedShardIds = new HashSet<>(); - final List requestLeases = leaseCaptor.getAllValues(); final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - validateHashRangeinLease(requestLeases); - assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); - assertThat(extendedSequenceNumbers.size(), equalTo(0)); + validateLeases(requestLeases); + assertEquals(0, extendedSequenceNumbers.size()); verify(shardDetector, never()).listShards(); verify(shardDetector, never()).listShardsWithoutConsumingResourceNotFoundException(); @@ -491,10 +467,10 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithEmptyLeaseTable() throws Exception { - final List shards = constructShardListForGraphA(); final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5")); - testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); + testCheckAndCreateLeaseForShardsIfMissing(SHARD_GRAPH_A, INITIAL_POSITION_TRIM_HORIZON, + expectedLeaseKeysToCreate); } /** @@ -511,10 +487,10 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable1() throws Exception { - final List shards = constructShardListForGraphA(); final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", "shardId-10")); - testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); + testCheckAndCreateLeaseForShardsIfMissing(SHARD_GRAPH_A, INITIAL_POSITION_AT_TIMESTAMP, + expectedLeaseKeysToCreate); } /** @@ -531,12 +507,11 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable2() throws Exception { - final List shards = constructShardListForGraphA(); final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-6", "shardId-7", "shardId-4", "shardId-5")); final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended .newInitialPositionAtTimestamp(new Date(200L)); - testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate); + testCheckAndCreateLeaseForShardsIfMissing(SHARD_GRAPH_A, initialPosition, expectedLeaseKeysToCreate); } /** @@ -553,10 +528,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() throws Exception { - final List shards = constructShardListForGraphA(); final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", "shardId-10")); - testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate); + testCheckAndCreateLeaseForShardsIfMissing(SHARD_GRAPH_A, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate); } /** @@ -574,7 +548,7 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithPartialLeaseTable() throws Exception { - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the // lease for shard-0 when reading from TRIM_HORIZON. final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); @@ -582,7 +556,7 @@ public class HierarchicalShardSyncerTest { .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER); - final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + final Set expectedLeaseKeysToCreate = Collections.singleton("shardId-0"); testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate, existingLeases); } @@ -601,7 +575,7 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable1() throws Exception { - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the // lease for shard-0 when reading from AT_TIMESTAMP. final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); @@ -609,7 +583,7 @@ public class HierarchicalShardSyncerTest { .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); - final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + final Set expectedLeaseKeysToCreate = Collections.singleton("shardId-0"); testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate, existingLeases); } @@ -628,7 +602,7 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable2() throws Exception { - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended .newInitialPositionAtTimestamp(new Date(200L)); // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the @@ -638,7 +612,7 @@ public class HierarchicalShardSyncerTest { .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); - final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + final Set expectedLeaseKeysToCreate = Collections.singleton("shardId-0"); testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate, existingLeases); } @@ -657,7 +631,7 @@ public class HierarchicalShardSyncerTest { */ @Test public void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() throws Exception { - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the // lease for shard-0 when reading from LATEST. final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); @@ -665,13 +639,13 @@ public class HierarchicalShardSyncerTest { .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, LEASE_OWNER); - final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + final Set expectedLeaseKeysToCreate = Collections.singleton("shardId-0"); testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate, existingLeases); } @Test(expected = KinesisClientLibIOException.class) public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpen() throws Exception { - final List shards = new ArrayList<>(constructShardListForGraphA()); + final List shards = new ArrayList<>(SHARD_GRAPH_A); final SequenceNumberRange range = shards.get(0).sequenceNumberRange().toBuilder().endingSequenceNumber(null) .build(); final Shard shard = shards.get(3).toBuilder().sequenceNumberRange(range).build(); @@ -691,7 +665,7 @@ public class HierarchicalShardSyncerTest { @Test(expected = KinesisClientLibIOException.class) public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenForMultiStream() throws Exception { - final List shards = new ArrayList<>(constructShardListForGraphA()); + final List shards = new ArrayList<>(SHARD_GRAPH_A); final SequenceNumberRange range = shards.get(0).sequenceNumberRange().toBuilder().endingSequenceNumber(null) .build(); final Shard shard = shards.get(3).toBuilder().sequenceNumberRange(range).build(); @@ -710,98 +684,40 @@ public class HierarchicalShardSyncerTest { } } + private void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren( + final String... expectedLeaseKeys) throws Exception { + final List shards = new ArrayList<>(SHARD_GRAPH_A); + final Shard shard = shards.get(5); + assertEquals("shardId-5", shard.shardId()); + + shards.remove(5); + + // shardId-5 in graph A has two children (shardId-9 and shardId-10). if shardId-5 + // is not closed, those children should be ignored when syncing shards, no leases + // should be obtained for them, and we should obtain a lease on the still-open + // parent. + shards.add(5, + shard.toBuilder() + .sequenceNumberRange(shard.sequenceNumberRange().toBuilder().endingSequenceNumber(null).build()) + .build()); + + testLeaseCreation(shards, true, expectedLeaseKeys); + } + /** * Test checkAndCreateLeasesForNewShards() when a parent is open and children of open parents are being ignored. */ @Test public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren() throws Exception { - final List shards = new ArrayList<>(constructShardListForGraphA()); - final Shard shard = shards.get(5); - assertThat(shard.shardId(), equalTo("shardId-5")); - - shards.remove(5); - - // shardId-5 in graph A has two children (shardId-9 and shardId-10). if shardId-5 - // is not closed, those children should be ignored when syncing shards, no leases - // should be obtained for them, and we should obtain a lease on the still-open - // parent. - shards.add(5, - shard.toBuilder() - .sequenceNumberRange(shard.sequenceNumberRange().toBuilder().endingSequenceNumber(null).build()) - .build()); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); - - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final List leases = leaseCaptor.getAllValues(); - final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set leaseSequenceNumbers = leases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - final Set expectedShardIds = new HashSet<>(Arrays.asList("shardId-4", "shardId-5", "shardId-8")); - - assertThat(leaseKeys.size(), equalTo(expectedShardIds.size())); - assertThat(leaseKeys, equalTo(expectedShardIds)); - assertThat(leaseSequenceNumbers.size(), equalTo(1)); - - leaseSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - - verify(shardDetector, never()).listShards(); - verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); - verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren( + "shardId-4", "shardId-5", "shardId-8"); } @Test public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildrenMultiStream() throws Exception { - final List shards = new ArrayList<>(constructShardListForGraphA()); - final Shard shard = shards.get(5); - assertThat(shard.shardId(), equalTo("shardId-5")); - - shards.remove(5); - - // shardId-5 in graph A has two children (shardId-9 and shardId-10). if shardId-5 - // is not closed, those children should be ignored when syncing shards, no leases - // should be obtained for them, and we should obtain a lease on the still-open - // parent. - shards.add(5, - shard.toBuilder() - .sequenceNumberRange(shard.sequenceNumberRange().toBuilder().endingSequenceNumber(null).build()) - .build()); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShardsWithoutConsumingResourceNotFoundException()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); setupMultiStream(); - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final List leases = leaseCaptor.getAllValues(); - final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set leaseSequenceNumbers = leases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - final Set expectedShardIds = new HashSet<>(toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-5", "shardId-8"))); - - assertThat(leaseKeys.size(), equalTo(expectedShardIds.size())); - assertThat(leaseKeys, equalTo(expectedShardIds)); - assertThat(leaseSequenceNumbers.size(), equalTo(1)); - - leaseSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); - - verify(shardDetector).listShardsWithoutConsumingResourceNotFoundException(); - verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren( + toMultiStreamLeases("shardId-4", "shardId-5", "shardId-8")); } @Test @@ -819,7 +735,7 @@ public class HierarchicalShardSyncerTest { private void testCheckAndCreateLeasesForNewShardsAndClosedShard(final ExtendedSequenceNumber sequenceNumber, final InitialPositionInStreamExtended position) throws Exception { final String shardIdPrefix = "shardId-%d"; - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; final List leases = createLeasesFromShards(shards, sequenceNumber, LEASE_OWNER); // Marking shardId-0 as ShardEnd. @@ -846,7 +762,7 @@ public class HierarchicalShardSyncerTest { final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); - assertThat(createLeases, equalTo(expectedCreateLeases)); + assertEquals(expectedCreateLeases, createLeases); verify(shardDetector, times(1)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -858,7 +774,7 @@ public class HierarchicalShardSyncerTest { SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); - assertThat(deleteLeases.size(), equalTo(0)); + assertTrue(deleteLeases.isEmpty()); verify(shardDetector, times(2)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); @@ -883,7 +799,7 @@ public class HierarchicalShardSyncerTest { final ExtendedSequenceNumber sequenceNumber, final InitialPositionInStreamExtended position) throws Exception { final String shardIdPrefix = "shardId-%d"; - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; final List leases = createLeasesFromShards(shards, sequenceNumber, LEASE_OWNER); // Marking shardId-0 as ShardEnd. @@ -933,11 +849,6 @@ public class HierarchicalShardSyncerTest { .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - final Set expectedShardIds = new HashSet<>( - Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - verify(shardDetector, times(3)).listShardsWithoutConsumingResourceNotFoundException(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); @@ -986,7 +897,7 @@ public class HierarchicalShardSyncerTest { final ExtendedSequenceNumber sequenceNumber, final InitialPositionInStreamExtended position) throws Exception { final String shardIdPrefix = "shardId-%d"; - final List shards = constructShardListForGraphA(); + final List shards = SHARD_GRAPH_A; final List leases = createLeasesFromShards(shards, sequenceNumber, LEASE_OWNER); // Marking shardId-0 as ShardEnd. @@ -1044,16 +955,6 @@ public class HierarchicalShardSyncerTest { } } - private Lease createLeaseFromShard(final Shard shard, final ExtendedSequenceNumber checkpoint, - final String leaseOwner) { - return createLeasesFromShards(Collections.singletonList(shard), checkpoint, leaseOwner).get(0); - } - - private MultiStreamLease createMultiStreamLeaseFromShard(final Shard shard, final ExtendedSequenceNumber checkpoint, - final String leaseOwner) { - return createMultiStreamLeasesFromShards(Collections.singletonList(shard), checkpoint, leaseOwner).get(0); - } - private List createLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, final String leaseOwner) { return shards.stream().map(shard -> { @@ -1136,9 +1037,9 @@ public class HierarchicalShardSyncerTest { final Set expectedSequenceNumbers = new HashSet<>(Collections .singletonList(new ExtendedSequenceNumber(initialPosition.getInitialPositionInStream().name()))); - assertThat(leases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(leaseSequenceNumbers, equalTo(expectedSequenceNumbers)); + assertEquals(expectedLeaseKeys.size(), leases.size()); + assertEquals(expectedLeaseKeys, leaseKeys); + assertEquals(expectedSequenceNumbers, leaseSequenceNumbers); verify(dynamoDBLeaseRefresher, times(expectedLeaseKeys.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); @@ -1161,12 +1062,12 @@ public class HierarchicalShardSyncerTest { final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); for (InitialPositionInStreamExtended initialPosition : initialPositions) { - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, + final List newLeases = determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, initialPosition); - assertThat(newLeases.size(), equalTo(2)); + assertEquals(2, newLeases.size()); for (Lease lease : newLeases) { - assertThat(expectedLeaseShardIds.contains(lease.leaseKey()), equalTo(true)); + assertTrue(expectedLeaseShardIds.contains(lease.leaseKey())); assertThat(lease.checkpoint(), equalTo(new ExtendedSequenceNumber(initialPosition.getInitialPositionInStream().toString()))); } @@ -1191,7 +1092,6 @@ public class HierarchicalShardSyncerTest { final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); - final Set inconsistentShardIds = Collections.emptySet(); Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); @@ -1199,7 +1099,7 @@ public class HierarchicalShardSyncerTest { final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, + final List newLeases = determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); assertThat(newLeases.size(), equalTo(1)); @@ -1222,12 +1122,12 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange1() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1246,11 +1146,11 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange2() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1269,14 +1169,14 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange3() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1295,11 +1195,11 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange4() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1319,14 +1219,14 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestC_PartialHashRange5() { - final List shards = constructShardListForGraphC(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-9", "shardId-10"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_C, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1345,10 +1245,8 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRange() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST); } /** @@ -1367,11 +1265,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeWithoutGC() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5", "shardId-6", "shardId-7"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST); } /** @@ -1390,14 +1286,13 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_EmptyLeaseTable() { - final List shards = constructShardListForGraphA(); - final List shardIdsOfCurrentLeases = Collections.emptyList(); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, Collections.emptyList(), INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1416,11 +1311,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeAcrossDifferentEpochs() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", "shardId-9", "shardId-10"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST); } /** @@ -1439,11 +1332,11 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_PartialHashRange() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-6"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1462,10 +1355,8 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRange() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-5"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST); } /** @@ -1484,11 +1375,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRangeWithoutGC() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST); } /** @@ -1507,12 +1396,11 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeLatestB_EmptyLeaseTable() { - final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Collections.emptyList(); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, Collections.emptyList(), INITIAL_POSITION_LATEST, + expectedShardIdCheckpointMap); } /** @@ -1531,13 +1419,13 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange1() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, + expectedShardIdCheckpointMap); } /** @@ -1556,12 +1444,12 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange2() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, + expectedShardIdCheckpointMap); } /** @@ -1580,13 +1468,13 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange3() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, + expectedShardIdCheckpointMap); } /** @@ -1605,14 +1493,14 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange4() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, + expectedShardIdCheckpointMap); } /** @@ -1631,10 +1519,8 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRange() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON); } /** @@ -1653,11 +1539,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeWithoutGC() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5", "shardId-6", "shardId-7"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON); } /** @@ -1676,8 +1560,6 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_EmptyLeaseTable() { - final List shards = constructShardListForGraphA(); - final List shardIdsOfCurrentLeases = Collections.emptyList(); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); @@ -1685,7 +1567,8 @@ public class HierarchicalShardSyncerTest { expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, Collections.emptyList(), INITIAL_POSITION_TRIM_HORIZON, + expectedShardIdCheckpointMap); } /** @@ -1704,11 +1587,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeAcrossDifferentEpochs() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", "shardId-9", "shardId-10"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON); } /** @@ -1751,10 +1632,8 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRange() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-5"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON); } /** @@ -1773,11 +1652,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRangeWithoutGC() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON); } /** @@ -1796,12 +1673,11 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeHorizonB_EmptyLeaseTable() { - final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Collections.emptyList(); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, Collections.emptyList(), INITIAL_POSITION_TRIM_HORIZON, + expectedShardIdCheckpointMap); } /** @@ -1820,13 +1696,13 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange1() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, + expectedShardIdCheckpointMap); } /** @@ -1845,12 +1721,12 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange2() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, + expectedShardIdCheckpointMap); } /** @@ -1869,13 +1745,13 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange3() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, + expectedShardIdCheckpointMap); } /** @@ -1894,14 +1770,14 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange4() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, + expectedShardIdCheckpointMap); } /** @@ -1920,10 +1796,8 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRange() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP); } /** @@ -1942,11 +1816,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeWithoutGC() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5", "shardId-6", "shardId-7"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP); } /** @@ -1965,8 +1837,6 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_EmptyLeaseTable() { - final List shards = constructShardListForGraphA(); - final List shardIdsOfCurrentLeases = Collections.emptyList(); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); @@ -1974,7 +1844,8 @@ public class HierarchicalShardSyncerTest { expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, Collections.emptyList(), INITIAL_POSITION_AT_TIMESTAMP, + expectedShardIdCheckpointMap); } /** @@ -1993,11 +1864,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeAcrossDifferentEpochs() { - final List shards = constructShardListForGraphA(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", "shardId-9", "shardId-10"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP); } /** @@ -2040,10 +1909,8 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRange() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Collections.singletonList("shardId-5"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP); } /** @@ -2062,11 +1929,9 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRangeWithoutGC() { - final List shards = constructShardListForGraphB(); final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5"); - final Map expectedNoNewLeases = Collections.emptyMap(); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP); } /** @@ -2085,12 +1950,18 @@ public class HierarchicalShardSyncerTest { */ @Test public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_EmptyLeaseTable() { - final List shards = constructShardListForGraphB(); - final List shardIdsOfCurrentLeases = Collections.emptyList(); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); - assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + assertExpectedLeasesAreCreated(SHARD_GRAPH_B, Collections.emptyList(), INITIAL_POSITION_AT_TIMESTAMP, + expectedShardIdCheckpointMap); + } + + private void assertExpectedLeasesAreCreated( + final List shards, + final List shardIdsOfCurrentLeases, + final InitialPositionInStreamExtended initialPosition) { + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, initialPosition, Collections.emptyMap()); } private void assertExpectedLeasesAreCreated(List shards, @@ -2108,7 +1979,7 @@ public class HierarchicalShardSyncerTest { final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + final List newLeases = determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, shards, currentLeases, initialPosition); assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); @@ -2130,7 +2001,7 @@ public class HierarchicalShardSyncerTest { * shards from epoch 206 (open - no ending sequenceNumber) *
*/ - private List constructShardListForGraphA() { + private static List constructShardListForGraphA() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("11", "205"); @@ -2226,7 +2097,7 @@ public class HierarchicalShardSyncerTest { * 1 4 7 10 * */ - private List constructShardListForGraphB() { + private static List constructShardListForGraphB() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("1000", "1049"); final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("1050", "1099"); final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("1100", "1149"); @@ -2263,7 +2134,7 @@ public class HierarchicalShardSyncerTest { * shards from epoch 206 (open - no ending sequenceNumber) * */ - private List constructShardListForGraphC() { + private static List constructShardListForGraphC() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("103", null); @@ -2302,9 +2173,9 @@ public class HierarchicalShardSyncerTest { public void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() { final MemoizationContext memoizationContext = new MemoizationContext(); - assertThat(HierarchicalShardSyncer + assertFalse(HierarchicalShardSyncer .checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST, null, null, - null, memoizationContext), equalTo(false)); + null, memoizationContext)); } /** @@ -2315,9 +2186,9 @@ public class HierarchicalShardSyncerTest { final String shardId = "shardId-trimmed"; final MemoizationContext memoizationContext = new MemoizationContext(); - assertThat(HierarchicalShardSyncer + assertFalse(HierarchicalShardSyncer .checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, null, - new HashMap<>(), null, memoizationContext), equalTo(false)); + new HashMap<>(), null, memoizationContext)); } /** @@ -2332,10 +2203,9 @@ public class HierarchicalShardSyncerTest { final Map kinesisShards = new HashMap<>(); kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, null, null, null)); - assertThat( + assertTrue( HierarchicalShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, - shardIdsOfCurrentLeases, kinesisShards, newLeaseMap, memoizationContext), equalTo(true)); - assertThat(newLeaseMap.isEmpty(), equalTo(true)); + shardIdsOfCurrentLeases, kinesisShards, newLeaseMap, memoizationContext)); } /** @@ -2355,10 +2225,9 @@ public class HierarchicalShardSyncerTest { kinesisShards.put(adjacentParentShardId, ShardObjectHelper.newShard(adjacentParentShardId, null, null, null)); kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null)); - assertThat( + assertFalse( HierarchicalShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, - shardIdsOfCurrentLeases, kinesisShards, newLeaseMap, memoizationContext), equalTo(false)); - assertThat(newLeaseMap.isEmpty(), equalTo(true)); + shardIdsOfCurrentLeases, kinesisShards, newLeaseMap, memoizationContext)); } /** @@ -3021,9 +2890,6 @@ public class HierarchicalShardSyncerTest { // /** * Helper method. - * - * @param shardId - * @return */ private static Lease newLease(final String shardId) { final Lease lease = new Lease(); From 04a121a81196ee04219768591ee2c2e3424457b0 Mon Sep 17 00:00:00 2001 From: noahbt <127256797+noahbt@users.noreply.github.com> Date: Tue, 14 Mar 2023 14:09:47 -0700 Subject: [PATCH 32/88] Add new metric to be emitted on lease creation (#1060) * Add new metric to be emitted on lease creation * Rebase changes from master --------- Co-authored-by: Noah Thomas --- .../kinesis/leases/HierarchicalShardSyncer.java | 9 ++++++--- .../amazon/kinesis/lifecycle/ShutdownTask.java | 17 ++++++++++++++--- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index e191e5d0..b71796d3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -160,16 +160,19 @@ public class HierarchicalShardSyncer { final Set createdLeases = new HashSet<>(); for (Lease lease : newLeasesToCreate) { - long startTime = System.currentTimeMillis(); + final long startTime = System.currentTimeMillis(); boolean success = false; try { if(leaseRefresher.createLeaseIfNotExists(lease)) { createdLeases.add(lease); } success = true; - } - finally { + } finally { MetricsUtil.addSuccessAndLatency(scope, "CreateLease", success, startTime, MetricsLevel.DETAILED); + if (lease.checkpoint() != null) { + final String metricName = lease.checkpoint().isSentinelCheckpoint() ? lease.checkpoint().sequenceNumber() : "SEQUENCE_NUMBER"; + MetricsUtil.addSuccess(scope, "CreateLease_" + metricName, true, MetricsLevel.DETAILED); + } } } log.info("{} - Newly created leases {}: {}", streamIdentifier, createdLeases.size(), createdLeases); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index c436f38a..0322c0e2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -181,7 +181,7 @@ public class ShutdownTask implements ConsumerTask { + " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner."); } if (!CollectionUtils.isNullOrEmpty(childShards)) { - createLeasesForChildShardsIfNotExist(); + createLeasesForChildShardsIfNotExist(scope); updateLeaseWithChildShards(currentShardLease); } final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, @@ -239,7 +239,7 @@ public class ShutdownTask implements ConsumerTask { } } - private void createLeasesForChildShardsIfNotExist() + private void createLeasesForChildShardsIfNotExist(MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException { // For child shard resulted from merge of two parent shards, verify if both the parents are either present or // not present in the lease table before creating the lease entry. @@ -272,7 +272,18 @@ public class ShutdownTask implements ConsumerTask { if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) { log.debug("{} - Shard {} - Attempting to create lease for child shard {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseKey); final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); - leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); + final long startTime = System.currentTimeMillis(); + boolean success = false; + try { + leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); + success = true; + } finally { + MetricsUtil.addSuccessAndLatency(scope, "CreateLease", success, startTime, MetricsLevel.DETAILED); + if (leaseToCreate.checkpoint() != null) { + final String metricName = leaseToCreate.checkpoint().isSentinelCheckpoint() ? leaseToCreate.checkpoint().sequenceNumber() : "SEQUENCE_NUMBER"; + MetricsUtil.addSuccess(scope, "CreateLease_" + metricName, true, MetricsLevel.DETAILED); + } + } log.info("{} - Shard {}: Created child shard lease: {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseToCreate); } From 0cbd74f6e7f715c80bd92a9c98c35df420db21e7 Mon Sep 17 00:00:00 2001 From: chenylee-aws <122478603+chenylee-aws@users.noreply.github.com> Date: Wed, 15 Mar 2023 12:34:32 -0700 Subject: [PATCH 33/88] Allow leader to learn new leases upon re-election to avoid unnecessary shardSyncs (#1063) --- .../amazon/kinesis/coordinator/Scheduler.java | 32 ++++--- .../kinesis/coordinator/SchedulerTest.java | 88 +++++++++++++++---- 2 files changed, 89 insertions(+), 31 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index dacb7ba1..49d271b2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -180,7 +180,10 @@ public class Scheduler implements Runnable { private final Object lock = new Object(); private final Stopwatch streamSyncWatch = Stopwatch.createUnstarted(); + private boolean leasesSyncedOnAppInit = false; + @Getter(AccessLevel.NONE) + private boolean shouldSyncLeases = true; /** * Used to ensure that only one requestedShutdown is in progress at a time. @@ -279,8 +282,6 @@ public class Scheduler implements Runnable { PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis(); this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis(); -// this.retryGetRecordsInSeconds = this.retrievalConfig.retryGetRecordsInSeconds(); -// this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool(); this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis(); this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts(); this.shardDetectorProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).shardDetector(); @@ -419,6 +420,8 @@ public class Scheduler implements Runnable { // check for new streams and sync with the scheduler state if (isLeader()) { checkAndSyncStreamShardsAndLeases(); + } else { + shouldSyncLeases = true; } logExecutorState(); @@ -426,7 +429,7 @@ public class Scheduler implements Runnable { Thread.sleep(shardConsumerDispatchPollIntervalMillis); } catch (Exception e) { log.error("Worker.run caught exception, sleeping for {} milli seconds!", - String.valueOf(shardConsumerDispatchPollIntervalMillis), e); + shardConsumerDispatchPollIntervalMillis, e); try { Thread.sleep(shardConsumerDispatchPollIntervalMillis); } catch (InterruptedException ex) { @@ -454,15 +457,18 @@ public class Scheduler implements Runnable { final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, MULTI_STREAM_TRACKER); try { - // This is done to ensure that we clean up the stale streams lingering in the lease table. - if (!leasesSyncedOnAppInit && isMultiStreamMode) { - final List leases = fetchMultiStreamLeases(); - syncStreamsFromLeaseTableOnAppInit(leases); - leasesSyncedOnAppInit = true; - } - final Map newStreamConfigMap = streamTracker.streamConfigList() .stream().collect(Collectors.toMap(StreamConfig::streamIdentifier, Function.identity())); + // This is done to ensure that we clean up the stale streams lingering in the lease table. + if (isMultiStreamMode && (shouldSyncLeases || !leasesSyncedOnAppInit)) { + // Skip updating the stream map due to no new stream since last sync + if (newStreamConfigMap.keySet().stream().anyMatch(s -> !currentStreamConfigMap.containsKey(s))) { + syncStreamsFromLeaseTableOnAppInit(fetchMultiStreamLeases()); + } + leasesSyncedOnAppInit = true; + shouldSyncLeases = false; + } + // For new streams discovered, do a shard sync and update the currentStreamConfigMap for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { @@ -581,12 +587,14 @@ public class Scheduler implements Runnable { return streamsSynced; } - @VisibleForTesting boolean shouldSyncStreamsNow() { + @VisibleForTesting + boolean shouldSyncStreamsNow() { return isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); } - @VisibleForTesting void syncStreamsFromLeaseTableOnAppInit(List leases) { + @VisibleForTesting + void syncStreamsFromLeaseTableOnAppInit(List leases) { leases.stream() .map(lease -> StreamIdentifier.multiStreamInstance(lease.streamIdentifier())) .filter(streamIdentifier -> !currentStreamConfigMap.containsKey(streamIdentifier)) diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index af0755a3..0336ac6a 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -39,7 +39,6 @@ import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrat import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Date; import java.util.HashMap; @@ -66,6 +65,7 @@ import org.mockito.Mock; import org.mockito.Spy; import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.stubbing.OngoingStubbing; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -780,9 +780,8 @@ public class SchedulerTest { public void testKinesisStaleDeletedStreamCleanup() throws ProvisionedThroughputException, InvalidStateException, DependencyException { List streamConfigList1 = createDummyStreamConfigList(1,6); List streamConfigList2 = createDummyStreamConfigList(1,4); - when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); - prepareForStaleDeletedStreamCleanupTests(); + prepareForStaleDeletedStreamCleanupTests(streamConfigList1, streamConfigList2); // when KCL starts it starts with tracking 5 stream assertEquals(Sets.newHashSet(streamConfigList1), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); @@ -817,27 +816,12 @@ public class SchedulerTest { } - private void prepareForStaleDeletedStreamCleanupTests() { - - when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { - @Override public Duration waitPeriodToDeleteFormerStreams() { - return Duration.ofDays(1); - } - }); - - retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) - .retrievalFactory(retrievalFactory); - scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, - metricsConfig, processorConfig, retrievalConfig)); - when(scheduler.shouldSyncStreamsNow()).thenReturn(true); - } // Tests validate that no cleanup of stream is done if its still tracked in multiStreamTracker @Test public void testKinesisStaleDeletedStreamNoCleanUpForTrackedStream() throws ProvisionedThroughputException, InvalidStateException, DependencyException { List streamConfigList1 = createDummyStreamConfigList(1,6); - when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1); - prepareForStaleDeletedStreamCleanupTests(); + prepareForStaleDeletedStreamCleanupTests(streamConfigList1); scheduler.deletedStreamListProvider().add(createDummyStreamConfig(3).streamIdentifier()); @@ -974,6 +958,72 @@ public class SchedulerTest { verify(rejectedTaskEvent, times(1)).accept(any()); } + @Test + public void testUpdateStreamMapIfMissingLatestStream() throws Exception { + prepareMultiStreamScheduler(createDummyStreamConfigList(1, 6)); + scheduler.checkAndSyncStreamShardsAndLeases(); + verify(scheduler).syncStreamsFromLeaseTableOnAppInit(any()); + } + + @Test + public void testNoDdbLookUpAsStreamMapContainsAllStreams() throws Exception { + final List streamConfigList = createDummyStreamConfigList(1, 6); + prepareMultiStreamScheduler(streamConfigList); + // Populate currentStreamConfigMap to simulate that the leader has the latest streams. + streamConfigList.forEach(s -> scheduler.currentStreamConfigMap().put(s.streamIdentifier(), s)); + scheduler.checkAndSyncStreamShardsAndLeases(); + verify(scheduler, never()).syncStreamsFromLeaseTableOnAppInit(any()); + } + + @Test + public void testNoDdbLookUpForNewStreamAsLeaderFlippedTheShardSyncFlags() throws Exception { + prepareMultiStreamScheduler(); + scheduler.checkAndSyncStreamShardsAndLeases(); + verify(scheduler, never()).syncStreamsFromLeaseTableOnAppInit(any()); + + final List streamConfigList = createDummyStreamConfigList(1, 6); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList); + scheduler.checkAndSyncStreamShardsAndLeases(); + + // Since the sync path has been executed once before the DDB sync flags should be flipped + // to prevent doing DDB lookups in the subsequent runs. + verify(scheduler, never()).syncStreamsFromLeaseTableOnAppInit(any()); + assertEquals(0, streamConfigList.stream() + .filter(s -> !scheduler.currentStreamConfigMap().containsKey(s.streamIdentifier())).count()); + } + + @SafeVarargs + private final void prepareMultiStreamScheduler(List... streamConfigs) { + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + if (streamConfigs.length > 0) { + stubMultiStreamTracker(streamConfigs); + } + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + } + + @SafeVarargs + private final void prepareForStaleDeletedStreamCleanupTests(List... streamConfigs) { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override + public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofDays(1); + } + }); + stubMultiStreamTracker(streamConfigs); + prepareMultiStreamScheduler(); + } + + @SafeVarargs + private final void stubMultiStreamTracker(List... streamConfigs) { + OngoingStubbing> stub = when(multiStreamTracker.streamConfigList()); + for (List streamConfig : streamConfigs) { + stub = stub.thenReturn(streamConfig); + } + } + /*private void runAndTestWorker(int numShards, int threadPoolSize) throws Exception { final int numberOfRecordsPerShard = 10; final String kinesisShardPrefix = "kinesis-0-"; From 0a3fcb07bd5a9f889885b42ef0db233816501540 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 15 Mar 2023 14:09:46 -0700 Subject: [PATCH 34/88] Bump aws-java-sdk.version from 1.12.370 to 1.12.405 (#1037) Bumps `aws-java-sdk.version` from 1.12.370 to 1.12.405. Updates `aws-java-sdk-core` from 1.12.370 to 1.12.405 - [Release notes](https://github.com/aws/aws-sdk-java/releases) - [Changelog](https://github.com/aws/aws-sdk-java/blob/master/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-java/compare/1.12.370...1.12.405) Updates `aws-java-sdk-sts` from 1.12.370 to 1.12.405 - [Release notes](https://github.com/aws/aws-sdk-java/releases) - [Changelog](https://github.com/aws/aws-sdk-java/blob/master/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-java/compare/1.12.370...1.12.405) --- updated-dependencies: - dependency-name: com.amazonaws:aws-java-sdk-core dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: com.amazonaws:aws-java-sdk-sts dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client-multilang/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 8ec7c5a7..0bc43453 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -28,7 +28,7 @@ amazon-kinesis-client-multilang - 1.12.370 + 1.12.405 From 7416f8b9158f24bbd1a7da69d1d4b10cc7a0d10f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 15 Mar 2023 14:11:05 -0700 Subject: [PATCH 35/88] Bump gsr.version from 1.1.13 to 1.1.14 (#1038) Bumps `gsr.version` from 1.1.13 to 1.1.14. Updates `schema-registry-serde` from 1.1.13 to 1.1.14 - [Release notes](https://github.com/awslabs/aws-glue-schema-registry/releases) - [Changelog](https://github.com/awslabs/aws-glue-schema-registry/blob/master/CHANGELOG.md) - [Commits](https://github.com/awslabs/aws-glue-schema-registry/compare/v1.1.13...v1.1.14) Updates `schema-registry-common` from 1.1.13 to 1.1.14 - [Release notes](https://github.com/awslabs/aws-glue-schema-registry/releases) - [Changelog](https://github.com/awslabs/aws-glue-schema-registry/blob/master/CHANGELOG.md) - [Commits](https://github.com/awslabs/aws-glue-schema-registry/compare/v1.1.13...v1.1.14) --- updated-dependencies: - dependency-name: software.amazon.glue:schema-registry-serde dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.glue:schema-registry-common dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 4fe771b7..dfdf43db 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -51,7 +51,7 @@ libsqlite4java ${project.build.directory}/test-lib 2.0.6 - 1.1.13 + 1.1.14 From bc71990beca1ce582c3ac8cc9f9372f01073dd45 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 15 Mar 2023 14:14:31 -0700 Subject: [PATCH 36/88] Bump maven-javadoc-plugin from 3.3.1 to 3.5.0 (#1046) Bumps [maven-javadoc-plugin](https://github.com/apache/maven-javadoc-plugin) from 3.3.1 to 3.5.0. - [Release notes](https://github.com/apache/maven-javadoc-plugin/releases) - [Commits](https://github.com/apache/maven-javadoc-plugin/compare/maven-javadoc-plugin-3.3.1...maven-javadoc-plugin-3.5.0) --- updated-dependencies: - dependency-name: org.apache.maven.plugins:maven-javadoc-plugin dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 0bc43453..88880c2a 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -143,7 +143,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.3.1 + 3.5.0 attach-javadocs diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index dfdf43db..07872125 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -299,7 +299,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.3.1 + 3.5.0 com.amazonaws.services.kinesis.producer.protobuf From 9ff99f01824086e05e05ea82e3ce23edb08ff073 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 15 Mar 2023 14:18:31 -0700 Subject: [PATCH 37/88] Bump awssdk.version from 2.19.31 to 2.20.8 (#1047) Bumps `awssdk.version` from 2.19.31 to 2.20.8. Updates `kinesis` from 2.19.31 to 2.20.8 Updates `dynamodb` from 2.19.31 to 2.20.8 Updates `cloudwatch` from 2.19.31 to 2.20.8 Updates `netty-nio-client` from 2.19.31 to 2.20.8 Updates `sts` from 2.19.31 to 2.20.8 --- updated-dependencies: - dependency-name: software.amazon.awssdk:kinesis dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: software.amazon.awssdk:dynamodb dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: software.amazon.awssdk:cloudwatch dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: software.amazon.awssdk:netty-nio-client dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: software.amazon.awssdk:sts dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d7b6d42a..0b630721 100644 --- a/pom.xml +++ b/pom.xml @@ -34,7 +34,7 @@ 2.4.6-SNAPSHOT - 2.19.31 + 2.20.8 From 0d5007c04c800840a7f63b39cb03698671eebf15 Mon Sep 17 00:00:00 2001 From: noahbt <127256797+noahbt@users.noreply.github.com> Date: Fri, 17 Mar 2023 13:21:13 -0700 Subject: [PATCH 38/88] Files updated for KCL release 2.4.7 (#1067) Co-authored-by: Noah Thomas --- CHANGELOG.md | 33 +++++++++++++++++++ README.md | 2 +- .../kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 4 files changed, 36 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 33391211..cb5f63b4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,39 @@ For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md) --- +### Release 2.4.7 (March 16, 2023) +* [#1063](https://github.com/awslabs/amazon-kinesis-client/pull/1063) Allow leader to learn new leases upon re-election to avoid unnecessary shardSyncs +* [#1060](https://github.com/awslabs/amazon-kinesis-client/pull/1060) Add new metric to be emitted on lease creation +* [#1057](https://github.com/awslabs/amazon-kinesis-client/pull/1057) Added more logging in `Scheduler` w.r.t. `StreamConfig`s. +* [#1059](https://github.com/awslabs/amazon-kinesis-client/pull/1059) DRY: simplification of `HierarchicalShardSyncerTest`. +* [#1062](https://github.com/awslabs/amazon-kinesis-client/pull/1062) Fixed retry storm in `PrefetchRecordsPublisher`. +* [#1061](https://github.com/awslabs/amazon-kinesis-client/pull/1061) Fixed NPE in `LeaseCleanupManager`. +* [#1056](https://github.com/awslabs/amazon-kinesis-client/pull/1056) Clean up in-memory state of deleted kinesis stream in MultiStreamMode +* [#1058](https://github.com/awslabs/amazon-kinesis-client/pull/1058) Documentation: added `
` tags so fixed-format diagrams aren't garbled.
+* [#1053](https://github.com/awslabs/amazon-kinesis-client/pull/1053) Exposed convenience method of `ExtendedSequenceNumber#isSentinelCheckpoint()`
+* [#1043](https://github.com/awslabs/amazon-kinesis-client/pull/1043) Removed a `.swp` file, and updated `.gitignore`.
+* [#1047](https://github.com/awslabs/amazon-kinesis-client/pull/1047) Upgrade awssdk.version from 2.19.31 to 2.20.8
+* [#1046](https://github.com/awslabs/amazon-kinesis-client/pull/1046) Upgrade maven-javadoc-plugin from 3.3.1 to 3.5.0
+* [#1038](https://github.com/awslabs/amazon-kinesis-client/pull/1038) Upgrade gsr.version from 1.1.13 to 1.1.14
+* [#1037](https://github.com/awslabs/amazon-kinesis-client/pull/1037) Upgrade aws-java-sdk.version from 1.12.370 to 1.12.405
+
+### Release 2.4.6 (February 21, 2023)
+* [#1041](https://github.com/awslabs/amazon-kinesis-client/pull/1041) Minor optimizations (e.g., calculate-once, put instead of get+put)
+* [#1035](https://github.com/awslabs/amazon-kinesis-client/pull/1035) Release Note updates to avoid duplication and bitrot (e.g., 1.x release
+* [#935](https://github.com/awslabs/amazon-kinesis-client/pull/935) Pass isAtShardEnd correctly to processRecords call
+* [#1040](https://github.com/awslabs/amazon-kinesis-client/pull/1040) Increased logging verbosity around lease management
+* [#1024](https://github.com/awslabs/amazon-kinesis-client/pull/1024) Added logging w.r.t. StreamConfig handling.
+* [#1034](https://github.com/awslabs/amazon-kinesis-client/pull/1034) Optimization: 9~15% improvement in KinesisDataFetcher wall-time
+* [#1045](https://github.com/awslabs/amazon-kinesis-client/pull/1045) Fixed duplication of project version in children pom.xml
+* [#956](https://github.com/awslabs/amazon-kinesis-client/pull/956) Fixed warning message typos
+* [#795](https://github.com/awslabs/amazon-kinesis-client/pull/795) Fixed log message spacing
+* [#740](https://github.com/awslabs/amazon-kinesis-client/pull/740) Fixed typo in Comment
+* [#1028](https://github.com/awslabs/amazon-kinesis-client/pull/1028) Refactored MultiStreamTracker to provide and enhance OOP for both
+* [#1027](https://github.com/awslabs/amazon-kinesis-client/pull/1027) Removed CHECKSTYLE:OFF toggles which can invite/obscure sub-par code.
+* [#1032](https://github.com/awslabs/amazon-kinesis-client/pull/1032) Upgrade rxjava from 3.1.5 to 3.1.6
+* [#1030](https://github.com/awslabs/amazon-kinesis-client/pull/1030) Upgrade awssdk.version from 2.19.2 to 2.19.31
+* [#1029](https://github.com/awslabs/amazon-kinesis-client/pull/1029) Upgrade slf4j-api from 2.0.0 to 2.0.6
+* [#1015](https://github.com/awslabs/amazon-kinesis-client/pull/1015) Upgrade protobuf-java from 3.21.5 to 3.21.12
 
 ### Release 2.4.5 (January 04, 2023)
 * [#1014](https://github.com/awslabs/amazon-kinesis-client/pull/1014) Use AFTER_SEQUENCE_NUMBER iterator type for expired iterator request
diff --git a/README.md b/README.md
index 90853c1f..a7272f73 100644
--- a/README.md
+++ b/README.md
@@ -50,7 +50,7 @@ The recommended way to use the KCL for Java is to consume it from Maven.
   
       software.amazon.kinesis
       amazon-kinesis-client
-      2.4.4
+      2.4.7
   
   ```
 
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
index f45fa80d..78ea7d03 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
@@ -49,7 +49,7 @@ public class RetrievalConfig {
      */
     public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java";
 
-    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.6-SNAPSHOT";
+    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.7";
 
     /**
      * Client used to make calls to Kinesis for records retrieval
diff --git a/pom.xml b/pom.xml
index 0b630721..972562a8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -33,7 +33,7 @@
   
 
   
-    2.4.6-SNAPSHOT
+    2.4.7
     2.20.8
   
 

From 9d07af403db9df906c649c5ec0adebf488dda620 Mon Sep 17 00:00:00 2001
From: furq-aws <127275086+furq-aws@users.noreply.github.com>
Date: Mon, 20 Mar 2023 14:00:05 -0700
Subject: [PATCH 39/88] Fix flaky InitializationWaitsWhenLeaseTableIsEmpty test
 (#1069)

Update misconfigured Scheduler MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS test value to be in sync with source value
---
 .../java/software/amazon/kinesis/coordinator/SchedulerTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java
index 0336ac6a..2a0a1dae 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java
@@ -123,7 +123,7 @@ public class SchedulerTest {
     private final String applicationName = "applicationName";
     private final String streamName = "streamName";
     private final String namespace = "testNamespace";
-    private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 5 * 1000L;
+    private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1000L;
     private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L;
     private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L;
 

From 26a692530d346e7cb0200869e946425c1c738dd4 Mon Sep 17 00:00:00 2001
From: noahbt <127256797+noahbt@users.noreply.github.com>
Date: Mon, 20 Mar 2023 16:03:36 -0700
Subject: [PATCH 40/88] Revert changes to pom property (#1077)

Co-authored-by: Noah Thomas 
---
 amazon-kinesis-client-multilang/pom.xml | 2 +-
 amazon-kinesis-client/pom.xml           | 2 +-
 pom.xml                                 | 3 +--
 3 files changed, 3 insertions(+), 4 deletions(-)

diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml
index 88880c2a..00ae031e 100644
--- a/amazon-kinesis-client-multilang/pom.xml
+++ b/amazon-kinesis-client-multilang/pom.xml
@@ -21,7 +21,7 @@
   
     amazon-kinesis-client-pom
     software.amazon.kinesis
-    ${revision}
+    2.4.7
   
   4.0.0
 
diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml
index 07872125..7e74d137 100644
--- a/amazon-kinesis-client/pom.xml
+++ b/amazon-kinesis-client/pom.xml
@@ -22,7 +22,7 @@
   
     software.amazon.kinesis
     amazon-kinesis-client-pom
-    ${revision}
+    2.4.7
   
 
   amazon-kinesis-client
diff --git a/pom.xml b/pom.xml
index 972562a8..e7014933 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,7 +22,7 @@
   amazon-kinesis-client-pom
   pom
   Amazon Kinesis Client Library
-  ${revision}
+  2.4.7
   The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
     from Amazon Kinesis.
   
@@ -33,7 +33,6 @@
   
 
   
-    2.4.7
     2.20.8
   
 

From 177303d5571e0f64a7dc9abec7f1294337854d61 Mon Sep 17 00:00:00 2001
From: noahbt <127256797+noahbt@users.noreply.github.com>
Date: Mon, 20 Mar 2023 16:21:54 -0700
Subject: [PATCH 41/88] Changelog updated for 2.4.7 (#1078)

Co-authored-by: Noah Thomas 
---
 CHANGELOG.md | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index cb5f63b4..fa9441ae 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -3,16 +3,18 @@
 For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md)
 
 ---
-### Release 2.4.7 (March 16, 2023)
+### Release 2.4.7 (March 20, 2023)
 * [#1063](https://github.com/awslabs/amazon-kinesis-client/pull/1063) Allow leader to learn new leases upon re-election to avoid unnecessary shardSyncs
 * [#1060](https://github.com/awslabs/amazon-kinesis-client/pull/1060) Add new metric to be emitted on lease creation
 * [#1057](https://github.com/awslabs/amazon-kinesis-client/pull/1057) Added more logging in `Scheduler` w.r.t. `StreamConfig`s.
 * [#1059](https://github.com/awslabs/amazon-kinesis-client/pull/1059) DRY: simplification of `HierarchicalShardSyncerTest`.
 * [#1062](https://github.com/awslabs/amazon-kinesis-client/pull/1062) Fixed retry storm in `PrefetchRecordsPublisher`.
 * [#1061](https://github.com/awslabs/amazon-kinesis-client/pull/1061) Fixed NPE in `LeaseCleanupManager`.
+* [#1069](https://github.com/awslabs/amazon-kinesis-client/pull/1069) Fixed flaky InitializationWaitsWhenLeaseTableIsEmpty test
 * [#1056](https://github.com/awslabs/amazon-kinesis-client/pull/1056) Clean up in-memory state of deleted kinesis stream in MultiStreamMode
 * [#1058](https://github.com/awslabs/amazon-kinesis-client/pull/1058) Documentation: added `
` tags so fixed-format diagrams aren't garbled.
 * [#1053](https://github.com/awslabs/amazon-kinesis-client/pull/1053) Exposed convenience method of `ExtendedSequenceNumber#isSentinelCheckpoint()`
+* [#1077](https://github.com/awslabs/amazon-kinesis-client/pull/1077) Reverted changes to pom property
 * [#1043](https://github.com/awslabs/amazon-kinesis-client/pull/1043) Removed a `.swp` file, and updated `.gitignore`.
 * [#1047](https://github.com/awslabs/amazon-kinesis-client/pull/1047) Upgrade awssdk.version from 2.19.31 to 2.20.8
 * [#1046](https://github.com/awslabs/amazon-kinesis-client/pull/1046) Upgrade maven-javadoc-plugin from 3.3.1 to 3.5.0

From 6be92dc4ef30b7d9745263508b039eadc93d4671 Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Tue, 21 Mar 2023 19:52:17 -0400
Subject: [PATCH 42/88] Added metrics in `ShutdownTask` for scenarios when
 parent leases are missing. (#1080)

+ optimizations in `ShutdownTask` (e.g., `Random` static instance,
eliminated over-used Function)
+ DRY+KISS on `ShutdownTaskTest`
+ deleted some dead code
---
 .../kinesis/lifecycle/ConsumerStates.java     |  13 -
 .../amazon/kinesis/lifecycle/ProcessTask.java |  22 --
 .../kinesis/lifecycle/ShardConsumer.java      |   8 +-
 .../kinesis/lifecycle/ShutdownTask.java       | 130 ++++---
 .../kinesis/lifecycle/ShutdownTaskTest.java   | 358 ++++++++----------
 5 files changed, 244 insertions(+), 287 deletions(-)

diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java
index c4a87082..8a0cd358 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java
@@ -93,24 +93,11 @@ class ConsumerStates {
         }
     }
 
-
     /**
      * The initial state that any {@link ShardConsumer} should start in.
      */
     static final ConsumerState INITIAL_STATE = ShardConsumerState.WAITING_ON_PARENT_SHARDS.consumerState();
 
-    private static ConsumerState shutdownStateFor(ShutdownReason reason) {
-        switch (reason) {
-        case REQUESTED:
-            return ShardConsumerState.SHUTDOWN_REQUESTED.consumerState();
-        case SHARD_END:
-        case LEASE_LOST:
-            return ShardConsumerState.SHUTTING_DOWN.consumerState();
-        default:
-            throw new IllegalArgumentException("Unknown reason: " + reason);
-        }
-    }
-
     /**
      * This is the initial state of a shard consumer. This causes the consumer to remain blocked until the all parent
      * shards have been completed.
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java
index 9f616b0d..c3f9523d 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java
@@ -243,28 +243,6 @@ public class ProcessTask implements ConsumerTask {
         return (!records.isEmpty()) || shouldCallProcessRecordsEvenForEmptyRecordList;
     }
 
-    /**
-     * Emits metrics, and sleeps if there are no records available
-     *
-     * @param startTimeMillis
-     *            the time when the task started
-     */
-    private void handleNoRecords(long startTimeMillis) {
-        log.debug("Kinesis didn't return any records for shard {}", shardInfoId);
-
-        long sleepTimeMillis = idleTimeInMilliseconds - (System.currentTimeMillis() - startTimeMillis);
-        if (sleepTimeMillis > 0) {
-            sleepTimeMillis = Math.max(sleepTimeMillis, idleTimeInMilliseconds);
-            try {
-                log.debug("Sleeping for {} ms since there were no new records in shard {}", sleepTimeMillis,
-                        shardInfoId);
-                Thread.sleep(sleepTimeMillis);
-            } catch (InterruptedException e) {
-                log.debug("ShardId {}: Sleep was interrupted", shardInfoId);
-            }
-        }
-    }
-
     @Override
     public TaskType taskType() {
         return taskType;
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java
index a575a953..4162ea81 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java
@@ -59,7 +59,13 @@ public class ShardConsumer {
     private final ShardConsumerArgument shardConsumerArgument;
     @NonNull
     private final Optional logWarningForTaskAfterMillis;
+
+    /**
+     * @deprecated unused; to be removed in a "major" version bump
+     */
+    @Deprecated
     private final Function taskMetricsDecorator;
+
     private final int bufferSize;
     private final TaskExecutionListener taskExecutionListener;
     private final String streamIdentifier;
@@ -179,7 +185,6 @@ public class ShardConsumer {
                 }
                 stateChangeFuture = initializeComplete();
             }
-
         } catch (InterruptedException e) {
             //
             // Ignored should be handled by scheduler
@@ -199,7 +204,6 @@ public class ShardConsumer {
                 throw (Error) t;
             }
         }
-
     }
 
     @VisibleForTesting
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java
index 0322c0e2..31bc8f88 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java
@@ -34,6 +34,7 @@ import software.amazon.kinesis.leases.HierarchicalShardSyncer;
 import software.amazon.kinesis.leases.Lease;
 import software.amazon.kinesis.leases.LeaseCleanupManager;
 import software.amazon.kinesis.leases.LeaseCoordinator;
+import software.amazon.kinesis.leases.LeaseRefresher;
 import software.amazon.kinesis.leases.ShardDetector;
 import software.amazon.kinesis.leases.ShardInfo;
 import software.amazon.kinesis.leases.UpdateField;
@@ -54,7 +55,6 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
 
 import java.util.Random;
 import java.util.Set;
-import java.util.function.Function;
 import java.util.stream.Collectors;
 
 /**
@@ -66,6 +66,14 @@ import java.util.stream.Collectors;
 public class ShutdownTask implements ConsumerTask {
     private static final String SHUTDOWN_TASK_OPERATION = "ShutdownTask";
     private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown";
+
+    /**
+     * Reusable, immutable {@link LeaseLostInput}.
+     */
+    private static final LeaseLostInput LEASE_LOST_INPUT = LeaseLostInput.builder().build();
+
+    private static final Random RANDOM = new Random();
+
     @VisibleForTesting
     static final int RETRY_RANDOM_MAX_RANGE = 30;
 
@@ -101,8 +109,6 @@ public class ShutdownTask implements ConsumerTask {
     @NonNull
     private final LeaseCleanupManager leaseCleanupManager;
 
-    private static final Function leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo);
-
     /*
      * Invokes ShardRecordProcessor shutdown() API.
      * (non-Javadoc)
@@ -114,61 +120,61 @@ public class ShutdownTask implements ConsumerTask {
         recordProcessorCheckpointer.checkpointer().operation(SHUTDOWN_TASK_OPERATION);
         final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHUTDOWN_TASK_OPERATION);
 
-        Exception exception;
-
+        final String leaseKey = ShardInfo.getLeaseKey(shardInfo);
         try {
             try {
                 log.debug("Invoking shutdown() for shard {} with childShards {}, concurrencyToken {}. Shutdown reason: {}",
-                        leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason);
+                        leaseKey, childShards, shardInfo.concurrencyToken(), reason);
 
                 final long startTime = System.currentTimeMillis();
-                final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo));
-                final Runnable leaseLostAction = () -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build());
+                final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKey);
+                final Runnable leaseLostAction = () -> shardRecordProcessor.leaseLost(LEASE_LOST_INPUT);
 
                 if (reason == ShutdownReason.SHARD_END) {
                     try {
-                        takeShardEndAction(currentShardLease, scope, startTime);
+                        takeShardEndAction(currentShardLease, leaseKey, scope, startTime);
                     } catch (InvalidStateException e) {
                         // If InvalidStateException happens, it indicates we have a non recoverable error in short term.
-                        // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry shutting down.
+                        // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow
+                        // other worker to take the lease and retry shutting down.
                         log.warn("Lease {}: Invalid state encountered while shutting down shardConsumer with SHARD_END reason. " +
-                                "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. ", leaseKeyProvider.apply(shardInfo), e);
-                        dropLease(currentShardLease);
-                        throwOnApplicationException(leaseLostAction, scope, startTime);
+                                "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason.",
+                                leaseKey, e);
+                        dropLease(currentShardLease, leaseKey);
+                        throwOnApplicationException(leaseKey, leaseLostAction, scope, startTime);
                     }
                 } else {
-                    throwOnApplicationException(leaseLostAction, scope, startTime);
+                    throwOnApplicationException(leaseKey, leaseLostAction, scope, startTime);
                 }
 
-                log.debug("Shutting down retrieval strategy for shard {}.", leaseKeyProvider.apply(shardInfo));
+                log.debug("Shutting down retrieval strategy for shard {}.", leaseKey);
                 recordsPublisher.shutdown();
-                log.debug("Record processor completed shutdown() for shard {}", leaseKeyProvider.apply(shardInfo));
+                log.debug("Record processor completed shutdown() for shard {}", leaseKey);
 
                 return new TaskResult(null);
             } catch (Exception e) {
                 if (e instanceof CustomerApplicationException) {
-                    log.error("Shard {}: Application exception. ", leaseKeyProvider.apply(shardInfo), e);
+                    log.error("Shard {}: Application exception.", leaseKey, e);
                 } else {
-                    log.error("Shard {}: Caught exception: ", leaseKeyProvider.apply(shardInfo), e);
+                    log.error("Shard {}: Caught exception:", leaseKey, e);
                 }
-                exception = e;
                 // backoff if we encounter an exception.
                 try {
                     Thread.sleep(this.backoffTimeMillis);
                 } catch (InterruptedException ie) {
-                    log.debug("Shard {}: Interrupted sleep", leaseKeyProvider.apply(shardInfo), ie);
+                    log.debug("Shard {}: Interrupted sleep", leaseKey, ie);
                 }
+
+                return new TaskResult(e);
             }
         } finally {
             MetricsUtil.endScope(scope);
         }
-
-        return new TaskResult(exception);
     }
 
     // Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup.
     private void takeShardEndAction(Lease currentShardLease,
-            MetricsScope scope, long startTime)
+            final String leaseKey, MetricsScope scope, long startTime)
             throws DependencyException, ProvisionedThroughputException, InvalidStateException,
             CustomerApplicationException {
         // Create new lease for the child shards if they don't exist.
@@ -177,7 +183,7 @@ public class ShutdownTask implements ConsumerTask {
         // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards.
         // This scenario could happen when customer deletes the stream while leaving the KCL application running.
         if (currentShardLease == null) {
-            throw new InvalidStateException(leaseKeyProvider.apply(shardInfo)
+            throw new InvalidStateException(leaseKey
                     + " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.");
         }
         if (!CollectionUtils.isNullOrEmpty(childShards)) {
@@ -189,7 +195,7 @@ public class ShutdownTask implements ConsumerTask {
         if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) {
             boolean isSuccess = false;
             try {
-                isSuccess = attemptShardEndCheckpointing(scope, startTime);
+                isSuccess = attemptShardEndCheckpointing(leaseKey, scope, startTime);
             } finally {
                 // Check if either the shard end ddb persist is successful or
                 // if childshards is empty. When child shards is empty then either it is due to
@@ -202,38 +208,41 @@ public class ShutdownTask implements ConsumerTask {
         }
     }
 
-    private boolean attemptShardEndCheckpointing(MetricsScope scope, long startTime)
+    private boolean attemptShardEndCheckpointing(final String leaseKey, MetricsScope scope, long startTime)
             throws DependencyException, ProvisionedThroughputException, InvalidStateException,
             CustomerApplicationException {
-        final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo)))
-                .orElseThrow(() -> new InvalidStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist."));
+        final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKey))
+                .orElseThrow(() -> new InvalidStateException("Lease for shard " + leaseKey + " does not exist."));
         if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) {
             // Call the shardRecordsProcessor to checkpoint with SHARD_END sequence number.
-            // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded.
-            throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime);
+            // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is
+            // successful after calling shardEnded.
+            throwOnApplicationException(leaseKey, () -> applicationCheckpointAndVerification(leaseKey),
+                    scope, startTime);
         }
         return true;
     }
 
-    private void applicationCheckpointAndVerification() {
+    private void applicationCheckpointAndVerification(final String leaseKey) {
         recordProcessorCheckpointer
                 .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue());
         recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END);
         shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
         final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue();
-        if (lastCheckpointValue == null
-                || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) {
+        if (!ExtendedSequenceNumber.SHARD_END.equals(lastCheckpointValue)) {
             throw new IllegalArgumentException("Application didn't checkpoint at end of shard "
-                    + leaseKeyProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " +
+                    + leaseKey + ". Application must checkpoint upon shard end. " +
                     "See ShardRecordProcessor.shardEnded javadocs for more information.");
         }
     }
 
-    private void throwOnApplicationException(Runnable action, MetricsScope metricsScope, final long startTime) throws CustomerApplicationException {
+    private void throwOnApplicationException(final String leaseKey, Runnable action, MetricsScope metricsScope,
+            final long startTime)
+            throws CustomerApplicationException {
         try {
             action.run();
         } catch (Exception e) {
-            throw new CustomerApplicationException("Customer application throws exception for shard " + leaseKeyProvider.apply(shardInfo) +": ", e);
+            throw new CustomerApplicationException("Customer application throws exception for shard " + leaseKey + ": ", e);
         } finally {
             MetricsUtil.addLatency(metricsScope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY);
         }
@@ -241,41 +250,48 @@ public class ShutdownTask implements ConsumerTask {
 
     private void createLeasesForChildShardsIfNotExist(MetricsScope scope)
             throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+        final LeaseRefresher leaseRefresher = leaseCoordinator.leaseRefresher();
+
         // 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) {
+        if (childShards.size() == 1) {
             final ChildShard childShard = childShards.get(0);
             final List parentLeaseKeys = childShard.parentShards().stream()
                     .map(parentShardId -> ShardInfo.getLeaseKey(shardInfo, parentShardId)).collect(Collectors.toList());
             if (parentLeaseKeys.size() != 2) {
+                MetricsUtil.addCount(scope, "MissingMergeParent", 1, MetricsLevel.SUMMARY);
                 throw new InvalidStateException("Shard " + shardInfo.shardId() + "'s only child shard " + childShard
                         + " does not contain other parent information.");
-            } else {
-                boolean isValidLeaseTableState =
-                        Objects.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(0))) == Objects
-                                .isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(1)));
-                if (!isValidLeaseTableState) {
-                    if (!isOneInNProbability(RETRY_RANDOM_MAX_RANGE)) {
-                        throw new BlockedOnParentShardException(
-                                "Shard " + shardInfo.shardId() + "'s only child shard " + childShard
-                                        + " has partial parent information in lease table. Hence deferring lease creation of child shard.");
-                    } else {
-                        throw new InvalidStateException(
-                                "Shard " + shardInfo.shardId() + "'s only child shard " + childShard
-                                        + " has partial parent information in lease table. Hence deferring lease creation of child shard.");
-                    }
+            }
+
+            final Lease parentLease0 = leaseRefresher.getLease(parentLeaseKeys.get(0));
+            final Lease parentLease1 = leaseRefresher.getLease(parentLeaseKeys.get(1));
+            if (Objects.isNull(parentLease0) != Objects.isNull(parentLease1)) {
+                MetricsUtil.addCount(scope, "MissingMergeParentLease", 1, MetricsLevel.SUMMARY);
+                final String message = "Shard " + shardInfo.shardId() + "'s only child shard " + childShard +
+                        " has partial parent information in lease table: [parent0=" + parentLease0 +
+                        ", parent1=" + parentLease1 + "]. Hence deferring lease creation of child shard.";
+                if (isOneInNProbability(RETRY_RANDOM_MAX_RANGE)) {
+                    // abort further attempts and drop the lease; lease will
+                    // be reassigned
+                    throw new InvalidStateException(message);
+                } else {
+                    // initiate a Thread.sleep(...) and keep the lease;
+                    // keeping the lease decreases churn of lease reassignments
+                    throw new BlockedOnParentShardException(message);
                 }
             }
         }
+
         for(ChildShard childShard : childShards) {
             final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId());
-            if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) {
+            if (leaseRefresher.getLease(leaseKey) == null) {
                 log.debug("{} - Shard {} - Attempting to create lease for child shard {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseKey);
                 final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier());
                 final long startTime = System.currentTimeMillis();
                 boolean success = false;
                 try {
-                    leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate);
+                    leaseRefresher.createLeaseIfNotExists(leaseToCreate);
                     success = true;
                 } finally {
                     MetricsUtil.addSuccessAndLatency(scope, "CreateLease", success, startTime, MetricsLevel.DETAILED);
@@ -295,8 +311,7 @@ public class ShutdownTask implements ConsumerTask {
      */
     @VisibleForTesting
     boolean isOneInNProbability(int n) {
-        Random r = new Random();
-        return 1 == r.nextInt((n - 1) + 1) + 1;
+        return 0 == RANDOM.nextInt(n);
     }
 
     private void updateLeaseWithChildShards(Lease currentLease)
@@ -324,10 +339,9 @@ public class ShutdownTask implements ConsumerTask {
         return reason;
     }
 
-    private void dropLease(Lease currentLease) {
+    private void dropLease(Lease currentLease, final String leaseKey) {
         if (currentLease == null) {
-            log.warn("Shard {}: Unable to find the lease for shard. Will shutdown the shardConsumer directly.", leaseKeyProvider.apply(shardInfo));
-            return;
+            log.warn("Shard {}: Unable to find the lease for shard. Will shutdown the shardConsumer directly.", leaseKey);
         } else {
             leaseCoordinator.dropLease(currentLease);
             log.info("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey());
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java
index 6617984d..5967ea2a 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java
@@ -18,24 +18,29 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.atLeast;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
-import static software.amazon.kinesis.lifecycle.ShutdownTask.RETRY_RANDOM_MAX_RANGE;
+import static software.amazon.kinesis.lifecycle.ShutdownReason.LEASE_LOST;
+import static software.amazon.kinesis.lifecycle.ShutdownReason.SHARD_END;
 
-import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
-import java.util.UUID;
+import java.util.Set;
 
-import com.google.common.collect.ImmutableList;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
 import org.mockito.Matchers;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
@@ -78,18 +83,19 @@ public class ShutdownTaskTest {
     private static final long TASK_BACKOFF_TIME_MILLIS = 1L;
     private static final InitialPositionInStreamExtended INITIAL_POSITION_TRIM_HORIZON =
             InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON);
-    private static final ShutdownReason SHARD_END_SHUTDOWN_REASON = ShutdownReason.SHARD_END;
-    private static final ShutdownReason LEASE_LOST_SHUTDOWN_REASON  = ShutdownReason.LEASE_LOST;
     private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory();
 
-    private final String concurrencyToken = "0-1-2-3-4";
-    private final String shardId = "shardId-0";
-    private boolean cleanupLeasesOfCompletedShards = false;
-    private boolean ignoreUnexpectedChildShards = false;
-    private ShardInfo shardInfo;
+    private static final StreamIdentifier STREAM_IDENTIFIER = StreamIdentifier.singleStreamInstance("streamName");
+
+    /**
+     * Shard id for the default-provided {@link ShardInfo} and {@link Lease}.
+     */
+    private static final String SHARD_ID = "shardId-0";
+    private static final ShardInfo SHARD_INFO = new ShardInfo(SHARD_ID, "concurrencyToken",
+            Collections.emptySet(), ExtendedSequenceNumber.LATEST);
+
     private ShutdownTask task;
-    private StreamIdentifier streamIdentifier = StreamIdentifier.singleStreamInstance("streamName");
-    
+
     @Mock
     private RecordsPublisher recordsPublisher;
     @Mock
@@ -111,20 +117,18 @@ public class ShutdownTaskTest {
 
     @Before
     public void setUp() throws Exception {
-        doNothing().when(recordsPublisher).shutdown();
         when(recordProcessorCheckpointer.checkpointer()).thenReturn(checkpointer);
+        when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END);
         final Lease childLease = new Lease();
         childLease.leaseKey("childShardLeaseKey");
         when(hierarchicalShardSyncer.createLeaseForChildShard(Matchers.any(ChildShard.class), Matchers.any(StreamIdentifier.class)))
                 .thenReturn(childLease);
+        setupLease(SHARD_ID, Collections.emptyList());
 
-        shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(),
-                ExtendedSequenceNumber.LATEST);
+        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
+        when(shardDetector.streamIdentifier()).thenReturn(STREAM_IDENTIFIER);
 
-        task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards(), streamIdentifier, leaseCleanupManager);
+        task = createShutdownTask(SHARD_END, constructChildrenFromSplit());
     }
 
     /**
@@ -132,13 +136,8 @@ public class ShutdownTaskTest {
      * This test is for the scenario that customer doesn't implement checkpoint in their implementation
      */
     @Test
-    public final void testCallWhenApplicationDoesNotCheckpoint() throws Exception {
+    public final void testCallWhenApplicationDoesNotCheckpoint() {
         when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298"));
-        Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId"), Collections.emptyList(), ExtendedSequenceNumber.LATEST);
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease);
-        when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease);
-        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
-        when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true);
 
         final TaskResult result = task.call();
         assertNotNull(result.getException());
@@ -151,17 +150,13 @@ public class ShutdownTaskTest {
      */
     @Test
     public final void testCallWhenCreatingNewLeasesThrows() throws Exception {
-        when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END);
-        Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId"));
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease);
-        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
         when(hierarchicalShardSyncer.createLeaseForChildShard(Matchers.any(ChildShard.class), Matchers.any(StreamIdentifier.class)))
                 .thenThrow(new InvalidStateException("InvalidStateException is thrown"));
 
         final TaskResult result = task.call();
         assertNull(result.getException());
         verify(recordsPublisher).shutdown();
-        verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
+        verify(shardRecordProcessor, never()).shardEnded(any(ShardEndedInput.class));
         verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build());
         verify(leaseCoordinator).dropLease(Matchers.any(Lease.class));
     }
@@ -172,145 +167,101 @@ public class ShutdownTaskTest {
      */
     @Test
     public final void testCallWhenTrueShardEnd() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-        shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
-                                  ExtendedSequenceNumber.LATEST);
-        task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                                SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                                ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                                hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards(), streamIdentifier, leaseCleanupManager);
-
-        when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END);
-        Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId"));
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease);
-        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
-        when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true);
-        when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease);
-
         final TaskResult result = task.call();
         assertNull(result.getException());
-        verify(recordsPublisher).shutdown();
+        verifyShutdownAndNoDrop();
         verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
-        verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build());
         verify(leaseRefresher).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class));
         verify(leaseRefresher, times(2)).createLeaseIfNotExists(Matchers.any(Lease.class));
-        verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class));
-        verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class));
+        verify(leaseCleanupManager).enqueueForDeletion(any(LeasePendingDeletion.class));
     }
 
+    /**
+     * Tests the scenario when one, but not both, parent shards are accessible.
+     * This test should drop the lease so another worker can make an attempt.
+     */
     @Test
-    public final void testCallThrowsUntilParentInfoNotPresentInLease() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-        shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
-                ExtendedSequenceNumber.LATEST);
-        when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END);
-        Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2"));
-        Lease parentLease = LeaseHelper.createLease("shardId-1", "leaseOwner", Collections.emptyList());
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease);
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-1"))
-                .thenReturn(null, null, null, null, null, parentLease);
-        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
-        when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true);
-        when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease);
-        // Return null lease first time to simulate partial parent lease info
-        when(leaseRefresher.getLease("shardId-1"))
-                .thenReturn(null, null, null, null, null, parentLease);
-
-        // Make first 5 attempts with partial parent info in lease table
-        for (int i = 0; i < 5; i++) {
-            ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                    SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                    ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                    hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager));
-            when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false);
-            TaskResult result = task.call();
-            assertNotNull(result.getException());
-            assertTrue(result.getException() instanceof BlockedOnParentShardException);
-            assertTrue(result.getException().getMessage().contains("has partial parent information in lease table"));
-            verify(recordsPublisher, never()).shutdown();
-            verify(shardRecordProcessor, never())
-                    .shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
-            verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build());
-            verify(leaseCoordinator, never())
-                    .updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString());
-            verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class));
-            verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE);
-            verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class));
-            verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class));
-        }
-
-        // make next attempt with complete parent info in lease table
-        ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager));
-        when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false);
-        TaskResult result = task.call();
-        assertNull(result.getException());
-        verify(recordsPublisher).shutdown();
-        verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
-        verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build());
-        verify(leaseRefresher).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class));
-        verify(leaseRefresher, times(1)).createLeaseIfNotExists(Matchers.any(Lease.class));
-        verify(task, never()).isOneInNProbability(RETRY_RANDOM_MAX_RANGE);
-        verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class));
-        verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class));
+    public void testMergeChildWhereOneParentHasLeaseAndInvalidState() throws Exception {
+        testMergeChildWhereOneParentHasLease(false);
     }
 
+    /**
+     * Tests the scenario when one, but not both, parent shards are accessible.
+     * This test should retain the lease.
+     */
     @Test
-    public final void testCallTriggersLeaseLossWhenParentInfoNotPresentInLease() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-        shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(),
-                ExtendedSequenceNumber.LATEST);
+    public void testMergeChildWhereOneParentHasLeaseAndBlockOnParent() throws Exception {
+        testMergeChildWhereOneParentHasLease(true);
+    }
 
-        when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END);
-        Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2"));
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease);
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-1"))
-                .thenReturn(null, null, null, null, null, null, null, null, null, null, null);
-        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
-        when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true);
-        when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease);
-        // Return null lease first time to simulate partial parent lease info
-        when(leaseRefresher.getLease("shardId-1"))
-                .thenReturn(null, null, null, null, null, null, null, null, null, null, null);
+    private void testMergeChildWhereOneParentHasLease(final boolean blockOnParent) throws Exception {
+        // the @Before setup makes the `SHARD_ID` parent accessible
+        final ChildShard mergeChild = constructChildFromMerge();
+        final TaskResult result = createShutdownTaskSpy(blockOnParent, Collections.singletonList(mergeChild)).call();
 
-        // Make first 10 attempts with partial parent info in lease table
-        for (int i = 0; i < 10; i++) {
-            ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                    SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                    ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                    hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager));
-            when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false);
-            TaskResult result = task.call();
+        if (blockOnParent) {
             assertNotNull(result.getException());
-            assertTrue(result.getException() instanceof BlockedOnParentShardException);
-            assertTrue(result.getException().getMessage().contains("has partial parent information in lease table"));
+            assertEquals(BlockedOnParentShardException.class, result.getException().getClass());
+
+            verify(leaseCoordinator, never()).dropLease(any(Lease.class));
+            verify(shardRecordProcessor, never()).leaseLost(any(LeaseLostInput.class));
             verify(recordsPublisher, never()).shutdown();
-            verify(shardRecordProcessor, never())
-                    .shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
-            verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build());
-            verify(leaseCoordinator, never())
-                    .updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString());
-            verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class));
-            verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE);
-            verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class));
-            verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class));
+        } else {
+            assertNull(result.getException());
+
+            // verify that only the accessible parent was dropped
+            final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class);
+            verify(leaseCoordinator).dropLease(leaseCaptor.capture());
+            assertEquals(SHARD_ID, leaseCaptor.getValue().leaseKey());
+
+            verify(shardRecordProcessor).leaseLost(any(LeaseLostInput.class));
+            verify(recordsPublisher).shutdown();
         }
 
-        // make final attempt with incomplete parent info in lease table
-        ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager));
-        when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(true);
-        TaskResult result = task.call();
-        assertNull(result.getException());
-        verify(recordsPublisher).shutdown();
-        verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
-        verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build());
-        verify(leaseRefresher, never()).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class));
-        verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class));
-        verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE);
-        verify(leaseCoordinator).dropLease(Matchers.any(Lease.class));
+        // verify that an attempt was made to retrieve both parents
+        final ArgumentCaptor leaseKeyCaptor = ArgumentCaptor.forClass(String.class);
+        verify(leaseRefresher, times(mergeChild.parentShards().size())).getLease(leaseKeyCaptor.capture());
+        assertEquals(mergeChild.parentShards(), leaseKeyCaptor.getAllValues());
+
         verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class));
+        verify(leaseRefresher, never()).updateLeaseWithMetaInfo(any(Lease.class), any(UpdateField.class));
+        verify(leaseRefresher, never()).createLeaseIfNotExists(any(Lease.class));
+        verify(shardRecordProcessor, never()).shardEnded(any(ShardEndedInput.class));
+    }
+
+    @Test
+    public final void testMergeChildWhereBothParentsHaveLeases() throws Exception {
+        // the @Before test setup makes the `SHARD_ID` parent accessible
+        final ChildShard mergeChild = constructChildFromMerge();
+        // make second parent accessible
+        setupLease(mergeChild.parentShards().get(1), Collections.emptyList());
+
+        final Lease mockChildLease = mock(Lease.class);
+        when(hierarchicalShardSyncer.createLeaseForChildShard(mergeChild, STREAM_IDENTIFIER))
+                .thenReturn(mockChildLease);
+
+        final TaskResult result = createShutdownTask(SHARD_END, Collections.singletonList(mergeChild)).call();
+
+        assertNull(result.getException());
+        verify(leaseCleanupManager).enqueueForDeletion(any(LeasePendingDeletion.class));
+
+        final ArgumentCaptor updateLeaseCaptor = ArgumentCaptor.forClass(Lease.class);
+        verify(leaseRefresher).updateLeaseWithMetaInfo(updateLeaseCaptor.capture(), eq(UpdateField.CHILD_SHARDS));
+        final Lease updatedLease = updateLeaseCaptor.getValue();
+        assertEquals(SHARD_ID, updatedLease.leaseKey());
+        assertEquals(Collections.singleton(mergeChild.shardId()), updatedLease.childShardIds());
+
+        verify(leaseRefresher).createLeaseIfNotExists(mockChildLease);
+
+        // verify all parent+child leases were retrieved
+        final Set expectedShardIds = new HashSet<>(mergeChild.parentShards());
+        expectedShardIds.add(mergeChild.shardId());
+        final ArgumentCaptor leaseKeyCaptor = ArgumentCaptor.forClass(String.class);
+        verify(leaseRefresher, atLeast(expectedShardIds.size())).getLease(leaseKeyCaptor.capture());
+        assertEquals(expectedShardIds, new HashSet<>(leaseKeyCaptor.getAllValues()));
+
+        verifyShutdownAndNoDrop();
+        verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
     }
 
     /**
@@ -319,25 +270,15 @@ public class ShutdownTaskTest {
      */
     @Test
     public final void testCallWhenShardNotFound() throws Exception {
-        final Lease heldLease = LeaseHelper.createLease("shardId-4", "leaseOwner", Collections.emptyList());
-        shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(),
-                                  ExtendedSequenceNumber.LATEST);
-        task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                                SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                                ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                                hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>(), streamIdentifier, leaseCleanupManager);
+        final Lease lease = setupLease("shardId-4", Collections.emptyList());
+        final ShardInfo shardInfo = new ShardInfo(lease.leaseKey(), "concurrencyToken", Collections.emptySet(),
+                ExtendedSequenceNumber.LATEST);
 
-        when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END);
-        when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher);
-        when(leaseRefresher.getLease("shardId-4")).thenReturn(heldLease);
-        when(leaseCoordinator.getCurrentlyHeldLease("shardId-4")).thenReturn(heldLease);
+        final TaskResult result = createShutdownTask(SHARD_END, Collections.emptyList(), shardInfo).call();
 
-        final TaskResult result = task.call();
         assertNull(result.getException());
-        verify(recordsPublisher).shutdown();
-        verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build());
-        verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class));
-        verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class));
+        verifyShutdownAndNoDrop();
+        verify(leaseRefresher, never()).createLeaseIfNotExists(any(Lease.class));
     }
 
     /**
@@ -346,14 +287,8 @@ public class ShutdownTaskTest {
      */
     @Test
     public final void testCallWhenLeaseLost() throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-        shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(),
-                ExtendedSequenceNumber.LATEST);
-        task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
-                LEASE_LOST_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards,
-                ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher,
-                hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>(), streamIdentifier, leaseCleanupManager);
+        final TaskResult result = createShutdownTask(LEASE_LOST, Collections.emptyList()).call();
 
-        final TaskResult result = task.call();
         assertNull(result.getException());
         verify(recordsPublisher).shutdown();
         verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build());
@@ -362,6 +297,17 @@ public class ShutdownTaskTest {
         verify(leaseRefresher, never()).createLeaseIfNotExists(any(Lease.class));
         verify(leaseCoordinator, never()).dropLease(any(Lease.class));
     }
+
+    @Test
+    public void testNullChildShards() throws Exception {
+        final TaskResult result = createShutdownTask(SHARD_END, null).call();
+
+        assertNull(result.getException());
+        verifyShutdownAndNoDrop();
+        verify(leaseCleanupManager).enqueueForDeletion(any(LeasePendingDeletion.class));
+        verify(leaseRefresher, never()).createLeaseIfNotExists(any(Lease.class));
+    }
+
     /**
      * Test method for {@link ShutdownTask#taskType()}.
      */
@@ -370,10 +316,24 @@ public class ShutdownTaskTest {
         assertEquals(TaskType.SHUTDOWN, task.taskType());
     }
 
-    private List constructChildShards() {
-        List childShards = new ArrayList<>();
-        List parentShards = new ArrayList<>();
-        parentShards.add(shardId);
+    private void verifyShutdownAndNoDrop() {
+        verify(recordsPublisher).shutdown();
+        verify(leaseCoordinator, never()).dropLease(any(Lease.class));
+        verify(shardRecordProcessor, never()).leaseLost(any(LeaseLostInput.class));
+    }
+
+    private Lease setupLease(final String leaseKey, final Collection parentShardIds) throws Exception {
+        final Lease lease = LeaseHelper.createLease(leaseKey, "leaseOwner", parentShardIds);
+        when(leaseCoordinator.getCurrentlyHeldLease(lease.leaseKey())).thenReturn(lease);
+        when(leaseRefresher.getLease(lease.leaseKey())).thenReturn(lease);
+        return lease;
+    }
+
+    /**
+     * Constructs two {@link ChildShard}s that mimic a shard split operation.
+     */
+    private List constructChildrenFromSplit() {
+        List parentShards = Collections.singletonList(SHARD_ID);
         ChildShard leftChild = ChildShard.builder()
                                          .shardId("ShardId-1")
                                          .parentShards(parentShards)
@@ -384,22 +344,36 @@ public class ShutdownTaskTest {
                                          .parentShards(parentShards)
                                          .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99"))
                                          .build();
-        childShards.add(leftChild);
-        childShards.add(rightChild);
-        return  childShards;
+        return Arrays.asList(leftChild, rightChild);
     }
 
-    private List constructChildShard() {
-        List childShards = new ArrayList<>();
-        List parentShards = new ArrayList<>();
-        parentShards.add(shardId);
-        parentShards.add("shardId-1");
-        ChildShard leftChild = ChildShard.builder()
+    /**
+     * Constructs a {@link ChildShard} that mimics a shard merge operation.
+     */
+    private ChildShard constructChildFromMerge() {
+        List parentShards = Arrays.asList(SHARD_ID, "shardId-1");
+        return ChildShard.builder()
                 .shardId("shardId-2")
                 .parentShards(parentShards)
                 .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49"))
                 .build();
-        childShards.add(leftChild);
-        return  childShards;
+    }
+
+    private ShutdownTask createShutdownTaskSpy(final boolean blockOnParent, final List childShards) {
+        final ShutdownTask spy = spy(createShutdownTask(SHARD_END, childShards));
+        when(spy.isOneInNProbability(ShutdownTask.RETRY_RANDOM_MAX_RANGE)).thenReturn(!blockOnParent);
+        return spy;
+    }
+
+    private ShutdownTask createShutdownTask(final ShutdownReason reason, final List childShards) {
+        return createShutdownTask(reason, childShards, SHARD_INFO);
+    }
+
+    private ShutdownTask createShutdownTask(final ShutdownReason reason, final List childShards,
+            final ShardInfo shardInfo) {
+        return new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer,
+                reason, INITIAL_POSITION_TRIM_HORIZON, false, false,
+                leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, hierarchicalShardSyncer,
+                NULL_METRICS_FACTORY, childShards, STREAM_IDENTIFIER, leaseCleanupManager);
     }
 }

From b894669bdaa5b1af91a1fef4835d4121fd3ef4c1 Mon Sep 17 00:00:00 2001
From: noahbt <127256797+noahbt@users.noreply.github.com>
Date: Tue, 21 Mar 2023 17:17:12 -0700
Subject: [PATCH 43/88] Changes made for 2.4.8 (#1083)

---
 CHANGELOG.md                                          | 11 ++++++++---
 README.md                                             |  2 +-
 amazon-kinesis-client-multilang/pom.xml               |  2 +-
 amazon-kinesis-client/pom.xml                         |  2 +-
 .../amazon/kinesis/retrieval/RetrievalConfig.java     |  2 +-
 pom.xml                                               |  2 +-
 6 files changed, 13 insertions(+), 8 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index fa9441ae..58f48a9f 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -3,18 +3,23 @@
 For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md)
 
 ---
-### Release 2.4.7 (March 20, 2023)
+### Release 2.4.8 (March 21, 2023)
+* [#1080](https://github.com/awslabs/amazon-kinesis-client/pull/1080) Added metric in `ShutdownTask` for scenario when parent leases are missing.
+* [#1077](https://github.com/awslabs/amazon-kinesis-client/pull/1077) Reverted changes to pom property
+* [#1069](https://github.com/awslabs/amazon-kinesis-client/pull/1069) Fixed flaky InitializationWaitsWhenLeaseTableIsEmpty test
+
+
+### Release 2.4.7 (March 17, 2023)
+* **NOTE: Due to an issue during the release process, the 2.4.7 published artifacts are incomplete and non-viable. Please use 2.4.8 or later.**
 * [#1063](https://github.com/awslabs/amazon-kinesis-client/pull/1063) Allow leader to learn new leases upon re-election to avoid unnecessary shardSyncs
 * [#1060](https://github.com/awslabs/amazon-kinesis-client/pull/1060) Add new metric to be emitted on lease creation
 * [#1057](https://github.com/awslabs/amazon-kinesis-client/pull/1057) Added more logging in `Scheduler` w.r.t. `StreamConfig`s.
 * [#1059](https://github.com/awslabs/amazon-kinesis-client/pull/1059) DRY: simplification of `HierarchicalShardSyncerTest`.
 * [#1062](https://github.com/awslabs/amazon-kinesis-client/pull/1062) Fixed retry storm in `PrefetchRecordsPublisher`.
 * [#1061](https://github.com/awslabs/amazon-kinesis-client/pull/1061) Fixed NPE in `LeaseCleanupManager`.
-* [#1069](https://github.com/awslabs/amazon-kinesis-client/pull/1069) Fixed flaky InitializationWaitsWhenLeaseTableIsEmpty test
 * [#1056](https://github.com/awslabs/amazon-kinesis-client/pull/1056) Clean up in-memory state of deleted kinesis stream in MultiStreamMode
 * [#1058](https://github.com/awslabs/amazon-kinesis-client/pull/1058) Documentation: added `
` tags so fixed-format diagrams aren't garbled.
 * [#1053](https://github.com/awslabs/amazon-kinesis-client/pull/1053) Exposed convenience method of `ExtendedSequenceNumber#isSentinelCheckpoint()`
-* [#1077](https://github.com/awslabs/amazon-kinesis-client/pull/1077) Reverted changes to pom property
 * [#1043](https://github.com/awslabs/amazon-kinesis-client/pull/1043) Removed a `.swp` file, and updated `.gitignore`.
 * [#1047](https://github.com/awslabs/amazon-kinesis-client/pull/1047) Upgrade awssdk.version from 2.19.31 to 2.20.8
 * [#1046](https://github.com/awslabs/amazon-kinesis-client/pull/1046) Upgrade maven-javadoc-plugin from 3.3.1 to 3.5.0
diff --git a/README.md b/README.md
index a7272f73..b2ebf3ef 100644
--- a/README.md
+++ b/README.md
@@ -50,7 +50,7 @@ The recommended way to use the KCL for Java is to consume it from Maven.
   
       software.amazon.kinesis
       amazon-kinesis-client
-      2.4.7
+      2.4.8
   
   ```
 
diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml
index 00ae031e..b76d1cb0 100644
--- a/amazon-kinesis-client-multilang/pom.xml
+++ b/amazon-kinesis-client-multilang/pom.xml
@@ -21,7 +21,7 @@
   
     amazon-kinesis-client-pom
     software.amazon.kinesis
-    2.4.7
+    2.4.8
   
   4.0.0
 
diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml
index 7e74d137..6614fd8b 100644
--- a/amazon-kinesis-client/pom.xml
+++ b/amazon-kinesis-client/pom.xml
@@ -22,7 +22,7 @@
   
     software.amazon.kinesis
     amazon-kinesis-client-pom
-    2.4.7
+    2.4.8
   
 
   amazon-kinesis-client
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
index 78ea7d03..496526ef 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
@@ -49,7 +49,7 @@ public class RetrievalConfig {
      */
     public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java";
 
-    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.7";
+    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.8";
 
     /**
      * Client used to make calls to Kinesis for records retrieval
diff --git a/pom.xml b/pom.xml
index e7014933..877d2d65 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,7 +22,7 @@
   amazon-kinesis-client-pom
   pom
   Amazon Kinesis Client Library
-  2.4.7
+  2.4.8
   The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
     from Amazon Kinesis.
   

From 0627ba50bbde852c7d48fc5eb5b51bcf285a37a5 Mon Sep 17 00:00:00 2001
From: furq-aws <127275086+furq-aws@users.noreply.github.com>
Date: Wed, 22 Mar 2023 09:33:28 -0700
Subject: [PATCH 44/88] Fix flaky HashRangesAreAlwaysComplete test (#1066)

Updates depth value for Merge and Reshard with some In-Progress Parents tests to prevent invalid hierarchy trees.
---
 .../kinesis/coordinator/PeriodicShardSyncManagerTest.java   | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java
index a2047a6b..9f8b34ac 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java
@@ -58,6 +58,8 @@ import static software.amazon.kinesis.leases.LeaseManagementConfig.DEFAULT_CONSE
 
 public class PeriodicShardSyncManagerTest {
 
+    private static final int MAX_DEPTH_WITH_IN_PROGRESS_PARENTS = 1;
+
     private StreamIdentifier streamIdentifier;
     private PeriodicShardSyncManager periodicShardSyncManager;
     @Mock
@@ -446,7 +448,7 @@ public class PeriodicShardSyncManagerTest {
         for (int i = 0; i < 1000; i++) {
             int maxInitialLeaseCount = 100;
             List leases = generateInitialLeases(maxInitialLeaseCount);
-            reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, true);
+            reshard(leases, MAX_DEPTH_WITH_IN_PROGRESS_PARENTS, ReshardType.MERGE, maxInitialLeaseCount, true);
             Collections.shuffle(leases);
             Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
         }
@@ -457,7 +459,7 @@ public class PeriodicShardSyncManagerTest {
         for (int i = 0; i < 1000; i++) {
             int maxInitialLeaseCount = 100;
             List leases = generateInitialLeases(maxInitialLeaseCount);
-            reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, true);
+            reshard(leases, MAX_DEPTH_WITH_IN_PROGRESS_PARENTS, ReshardType.ANY, maxInitialLeaseCount, true);
             Collections.shuffle(leases);
             Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent());
         }

From 2ecd1c4ac59e05148a5662284e4a74e3a6151b7d Mon Sep 17 00:00:00 2001
From: Ryan French 
Date: Thu, 23 Mar 2023 17:26:07 +0000
Subject: [PATCH 45/88] Allow tags to be added when lease table is created
 (#1065)

* Allow tags to be added when lease table is created

* Add javadoc comment to new member variable

* DRY up creation of the CreateTable Request builder

* Fix compiler error

* Remove unnecessary eq functions

* Fix indentation

* Add patch
---
 .../kinesis/leases/LeaseManagementConfig.java |  14 ++-
 .../DynamoDBLeaseManagementFactory.java       | 119 +++++++++++++++++-
 .../dynamodb/DynamoDBLeaseRefresher.java      |  55 +++++---
 .../dynamodb/DynamoDBLeaseRefresherTest.java  |  38 ++++++
 4 files changed, 207 insertions(+), 19 deletions(-)

diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java
index 4f2d3a2b..d80799fa 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java
@@ -18,6 +18,7 @@ package software.amazon.kinesis.leases;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import java.time.Duration;
+import java.util.Collection;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
@@ -28,8 +29,11 @@ import lombok.Data;
 import lombok.NonNull;
 import lombok.experimental.Accessors;
 import org.apache.commons.lang3.Validate;
+
+import software.amazon.awssdk.core.util.DefaultSdkAutoConstructList;
 import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
 import software.amazon.awssdk.services.dynamodb.model.BillingMode;
+import software.amazon.awssdk.services.dynamodb.model.Tag;
 import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
 import software.amazon.kinesis.common.InitialPositionInStream;
 import software.amazon.kinesis.common.InitialPositionInStreamExtended;
@@ -190,6 +194,13 @@ public class LeaseManagementConfig {
 
     private BillingMode billingMode = BillingMode.PAY_PER_REQUEST;
 
+    /**
+     * The list of tags to be applied to the DynamoDB table created for lease management.
+     *
+     * 

Default value: {@link DefaultSdkAutoConstructList} + */ + private Collection tags = DefaultSdkAutoConstructList.getInstance(); + /** * Frequency (in millis) of the auditor job to scan for partial leases in the lease table. * If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on @@ -333,7 +344,7 @@ public class LeaseManagementConfig { initialLeaseTableReadCapacity(), initialLeaseTableWriteCapacity(), hierarchicalShardSyncer(), - tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode()); + tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode(), tags()); } return leaseManagementFactory; } @@ -371,6 +382,7 @@ public class LeaseManagementConfig { tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode(), + tags(), leaseSerializer, customShardDetectorProvider(), isMultiStreamingMode, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java index 6bf2ff39..8b60f6dd 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -16,13 +16,16 @@ package software.amazon.kinesis.leases.dynamodb; import java.time.Duration; +import java.util.Collection; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.function.Function; import lombok.Data; import lombok.NonNull; +import software.amazon.awssdk.core.util.DefaultSdkAutoConstructList; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.Tag; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -86,6 +89,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { private final TableCreatorCallback tableCreatorCallback; private final Duration dynamoDbRequestTimeout; private final BillingMode billingMode; + private final Collection tags; private final boolean isMultiStreamMode; private final LeaseCleanupConfig leaseCleanupConfig; @@ -343,6 +347,61 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); } + /** + * Constructor. + * + * @param kinesisClient + * @param streamName + * @param dynamoDBClient + * @param tableName + * @param workerIdentifier + * @param executorService + * @param initialPositionInStream + * @param failoverTimeMillis + * @param epsilonMillis + * @param maxLeasesForWorker + * @param maxLeasesToStealAtOneTime + * @param maxLeaseRenewalThreads + * @param cleanupLeasesUponShardCompletion + * @param ignoreUnexpectedChildShards + * @param shardSyncIntervalMillis + * @param consistentReads + * @param listShardsBackoffTimeMillis + * @param maxListShardsRetryAttempts + * @param maxCacheMissesBeforeReload + * @param listShardsCacheAllowedAgeInSeconds + * @param cacheMissWarningModulus + * @param initialLeaseTableReadCapacity + * @param initialLeaseTableWriteCapacity + * @param hierarchicalShardSyncer + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + * @param tags + */ + @Deprecated + public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, + final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, + final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, + final long failoverTimeMillis, final long epsilonMillis, final int maxLeasesForWorker, + final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, + final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, + final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, + final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, + final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, + final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, + final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + Duration dynamoDbRequestTimeout, BillingMode billingMode, Collection tags) { + + this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, + cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); + } + /** * Constructor. * @@ -373,6 +432,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param dynamoDbRequestTimeout * @param billingMode */ + @Deprecated private DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final StreamConfig streamConfig, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, @@ -384,13 +444,65 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { + this(kinesisClient, streamConfig, dynamoDBClient, tableName, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, + cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, + deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, + DefaultSdkAutoConstructList.getInstance(), leaseSerializer); + } + + /** + * Constructor. + * + * @param kinesisClient + * @param streamConfig + * @param dynamoDBClient + * @param tableName + * @param workerIdentifier + * @param executorService + * @param failoverTimeMillis + * @param epsilonMillis + * @param maxLeasesForWorker + * @param maxLeasesToStealAtOneTime + * @param maxLeaseRenewalThreads + * @param cleanupLeasesUponShardCompletion + * @param ignoreUnexpectedChildShards + * @param shardSyncIntervalMillis + * @param consistentReads + * @param listShardsBackoffTimeMillis + * @param maxListShardsRetryAttempts + * @param maxCacheMissesBeforeReload + * @param listShardsCacheAllowedAgeInSeconds + * @param cacheMissWarningModulus + * @param initialLeaseTableReadCapacity + * @param initialLeaseTableWriteCapacity + * @param deprecatedHierarchicalShardSyncer + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + * @param tags + */ + private DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final StreamConfig streamConfig, + final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, + final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, + final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, + final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, + final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, + final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, + final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, + final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + Duration dynamoDbRequestTimeout, BillingMode billingMode, Collection tags, LeaseSerializer leaseSerializer) { this(kinesisClient, dynamoDBClient, tableName, workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer, + deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, tags, leaseSerializer, null, false, LeaseManagementConfig.DEFAULT_LEASE_CLEANUP_CONFIG); this.streamConfig = streamConfig; } @@ -437,7 +549,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, - Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer, + Duration dynamoDbRequestTimeout, BillingMode billingMode, Collection tags, LeaseSerializer leaseSerializer, Function customShardDetectorProvider, boolean isMultiStreamMode, LeaseCleanupConfig leaseCleanupConfig) { this.kinesisClient = kinesisClient; @@ -469,6 +581,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.customShardDetectorProvider = customShardDetectorProvider; this.isMultiStreamMode = isMultiStreamMode; this.leaseCleanupConfig = leaseCleanupConfig; + this.tags = tags; } @Override @@ -535,7 +648,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @Override public DynamoDBLeaseRefresher createLeaseRefresher() { return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, leaseSerializer, consistentReads, - tableCreatorCallback, dynamoDbRequestTimeout, billingMode); + tableCreatorCallback, dynamoDbRequestTimeout, billingMode, tags); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java index acb61a38..a6887f40 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableMap; import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -25,6 +26,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.core.util.DefaultSdkAutoConstructList; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; @@ -46,6 +48,7 @@ import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; import software.amazon.awssdk.services.dynamodb.model.ScanRequest; import software.amazon.awssdk.services.dynamodb.model.ScanResponse; import software.amazon.awssdk.services.dynamodb.model.TableStatus; +import software.amazon.awssdk.services.dynamodb.model.Tag; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -77,6 +80,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { private final Duration dynamoDbRequestTimeout; private final BillingMode billingMode; + private final Collection tags; private boolean newTableCreated = false; @@ -143,10 +147,29 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { * @param dynamoDbRequestTimeout * @param billingMode */ + @Deprecated public DynamoDBLeaseRefresher(final String table, final DynamoDbAsyncClient dynamoDBClient, final LeaseSerializer serializer, final boolean consistentReads, @NonNull final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, final BillingMode billingMode) { + this(table, dynamoDBClient, serializer, consistentReads, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, DefaultSdkAutoConstructList.getInstance()); + } + + /** + * Constructor. + * @param table + * @param dynamoDBClient + * @param serializer + * @param consistentReads + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + * @param tags + */ + public DynamoDBLeaseRefresher(final String table, final DynamoDbAsyncClient dynamoDBClient, + final LeaseSerializer serializer, final boolean consistentReads, + @NonNull final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, + final BillingMode billingMode, final Collection tags) { this.table = table; this.dynamoDBClient = dynamoDBClient; this.serializer = serializer; @@ -154,6 +177,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { this.tableCreatorCallback = tableCreatorCallback; this.dynamoDbRequestTimeout = dynamoDbRequestTimeout; this.billingMode = billingMode; + this.tags = tags; } /** @@ -162,20 +186,13 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean createLeaseTableIfNotExists(@NonNull final Long readCapacity, @NonNull final Long writeCapacity) throws ProvisionedThroughputException, DependencyException { - ProvisionedThroughput throughput = ProvisionedThroughput.builder().readCapacityUnits(readCapacity) + final CreateTableRequest.Builder builder = createTableRequestBuilder(); + if(BillingMode.PROVISIONED.equals(billingMode)) { + ProvisionedThroughput throughput = ProvisionedThroughput.builder().readCapacityUnits(readCapacity) .writeCapacityUnits(writeCapacity).build(); - final CreateTableRequest request; - if(BillingMode.PAY_PER_REQUEST.equals(billingMode)){ - request = CreateTableRequest.builder().tableName(table).keySchema(serializer.getKeySchema()) - .attributeDefinitions(serializer.getAttributeDefinitions()) - .billingMode(billingMode).build(); - } else { - request = CreateTableRequest.builder().tableName(table).keySchema(serializer.getKeySchema()) - .attributeDefinitions(serializer.getAttributeDefinitions()).provisionedThroughput(throughput) - .build(); + builder.provisionedThroughput(throughput); } - - return createTableIfNotExists(request); + return createTableIfNotExists(builder.build()); } /** @@ -184,9 +201,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean createLeaseTableIfNotExists() throws ProvisionedThroughputException, DependencyException { - final CreateTableRequest request = CreateTableRequest.builder().tableName(table).keySchema(serializer.getKeySchema()) - .attributeDefinitions(serializer.getAttributeDefinitions()) - .billingMode(billingMode).build(); + final CreateTableRequest request = createTableRequestBuilder().build(); return createTableIfNotExists(request); } @@ -787,6 +802,16 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } } + private CreateTableRequest.Builder createTableRequestBuilder() { + final CreateTableRequest.Builder builder = CreateTableRequest.builder().tableName(table).keySchema(serializer.getKeySchema()) + .attributeDefinitions(serializer.getAttributeDefinitions()) + .tags(tags); + if (BillingMode.PAY_PER_REQUEST.equals(billingMode)) { + builder.billingMode(billingMode); + } + return builder; + } + private AWSExceptionManager createExceptionManager() { final AWSExceptionManager exceptionManager = new AWSExceptionManager(); exceptionManager.add(DynamoDbException.class, t -> t); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java index ac814d75..643cc99c 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java @@ -26,6 +26,8 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -66,6 +68,7 @@ import software.amazon.awssdk.services.dynamodb.model.TableDescription; import software.amazon.awssdk.services.dynamodb.model.TableStatus; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; import software.amazon.awssdk.services.dynamodb.model.UpdateItemResponse; +import software.amazon.awssdk.services.dynamodb.model.Tag; import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.kinesis.leases.Lease; @@ -109,6 +112,7 @@ public class DynamoDBLeaseRefresherTest { private DynamoDBLeaseRefresher leaseRefresher; private DescribeTableRequest describeTableRequest; private CreateTableRequest createTableRequest; + private Collection tags; private Map serializedLease; @@ -313,6 +317,40 @@ public class DynamoDBLeaseRefresherTest { Assert.assertTrue(result); } + @Test + public void testCreateLeaseTableWithTagsIfNotExists() throws Exception { + tags = Collections.singletonList(Tag.builder().key("foo").value("bar").build()); + leaseRefresher = new DynamoDBLeaseRefresher(TABLE_NAME, dynamoDbClient, leaseSerializer, CONSISTENT_READS, + tableCreatorCallback, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT, BillingMode.PROVISIONED, tags); + + when(dynamoDbClient.describeTable(describeTableRequest)).thenReturn(mockDescribeTableFuture); + when(mockDescribeTableFuture.get(LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS)) + .thenThrow(ResourceNotFoundException.builder().message("Table doesn't exist").build()); + + final ProvisionedThroughput throughput = ProvisionedThroughput.builder().readCapacityUnits(10L) + .writeCapacityUnits(10L).build(); + final CreateTableRequest createTableRequest = CreateTableRequest.builder() + .tableName(TABLE_NAME) + .keySchema(leaseSerializer.getKeySchema()) + .attributeDefinitions(leaseSerializer.getAttributeDefinitions()) + .provisionedThroughput(throughput) + .tags(tags) + .build(); + when(dynamoDbClient.createTable(createTableRequest)).thenReturn(mockCreateTableFuture); + when(mockCreateTableFuture.get(LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS)) + .thenReturn(null); + + final boolean result = leaseRefresher.createLeaseTableIfNotExists(10L, 10L); + + verify(dynamoDbClient, times(1)).describeTable(describeTableRequest); + verify(dynamoDbClient, times(1)).createTable(createTableRequest); + verify(mockDescribeTableFuture, times(1)) + .get(LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + verify(mockCreateTableFuture, times(1)) + .get(LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + Assert.assertTrue(result); + } + @Test public void testCreateLeaseTableIfNotExists() throws Exception { when(dynamoDbClient.describeTable(describeTableRequest)).thenReturn(mockDescribeTableFuture); From 10cdf43b9da96dd65c94c4e1db5b73b4bef8aa49 Mon Sep 17 00:00:00 2001 From: noahbt <127256797+noahbt@users.noreply.github.com> Date: Thu, 23 Mar 2023 10:54:50 -0700 Subject: [PATCH 46/88] Updated versions to SNAPSHOT (#1085) --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../java/software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index b76d1cb0..6290ed1c 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.8 + 2.4.9-SNAPSHOT 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 6614fd8b..3f17aa58 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.8 + 2.4.9-SNAPSHOT amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 496526ef..3f001057 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.8"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.9-SNAPSHOT"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index 877d2d65..87ece5e2 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.8 + 2.4.9-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From b8d3390bf35c6847218b55063dec9197c6ccaaaa Mon Sep 17 00:00:00 2001 From: furq-aws <127275086+furq-aws@users.noreply.github.com> Date: Fri, 24 Mar 2023 04:53:44 -0700 Subject: [PATCH 47/88] Fix flaky restartAfterRequestTimerExpires tests (#1084) Add wait to allow subscriptions to start. This eliminates flakiness of tests restartAfterRequestTimerExpiresWhenNotGettingRecordsAfterInitialization() and restartAfterRequestTimerExpiresWhenInitialTaskExecutionIsRejected(). --- .../ShardConsumerSubscriberTest.java | 57 ++++++++----------- 1 file changed, 24 insertions(+), 33 deletions(-) diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java index 78e09fa1..09ba6ec9 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java @@ -79,6 +79,8 @@ public class ShardConsumerSubscriberTest { private static final String TERMINAL_MARKER = "Terminal"; + private static final long DEFAULT_NOTIFIER_TIMEOUT = 5000L; + private final RequestDetails lastSuccessfulRequestDetails = new RequestDetails(); @Mock @@ -125,10 +127,7 @@ public class ShardConsumerSubscriberTest { setupNotifierAnswer(1); - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - processedNotifier.wait(5000); - } + startSubscriptionsAndWait(); verify(shardConsumer).handleInput(argThat(eqProcessRecordsInput(processRecordsInput)), any(Subscription.class)); } @@ -139,10 +138,7 @@ public class ShardConsumerSubscriberTest { setupNotifierAnswer(recordsPublisher.responses.size()); - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - processedNotifier.wait(5000); - } + startSubscriptionsAndWait(); verify(shardConsumer, times(100)).handleInput(argThat(eqProcessRecordsInput(processRecordsInput)), any(Subscription.class)); } @@ -171,10 +167,7 @@ public class ShardConsumerSubscriberTest { } }).when(shardConsumer).handleInput(any(ProcessRecordsInput.class), any(Subscription.class)); - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - processedNotifier.wait(5000); - } + startSubscriptionsAndWait(); assertThat(subscriber.getAndResetDispatchFailure(), equalTo(testException)); assertThat(subscriber.getAndResetDispatchFailure(), nullValue()); @@ -192,10 +185,7 @@ public class ShardConsumerSubscriberTest { setupNotifierAnswer(10); - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - processedNotifier.wait(5000); - } + startSubscriptionsAndWait(); for (int attempts = 0; attempts < 10; attempts++) { if (subscriber.retrievalFailure() != null) { @@ -220,10 +210,7 @@ public class ShardConsumerSubscriberTest { setupNotifierAnswer(10); - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - processedNotifier.wait(5000); - } + startSubscriptionsAndWait(); for (int attempts = 0; attempts < 10; attempts++) { if (subscriber.retrievalFailure() != null) { @@ -236,7 +223,7 @@ public class ShardConsumerSubscriberTest { synchronized (processedNotifier) { assertThat(subscriber.healthCheck(100000), equalTo(expected)); - processedNotifier.wait(5000); + processedNotifier.wait(DEFAULT_NOTIFIER_TIMEOUT); } assertThat(recordsPublisher.restartedFrom, equalTo(edgeRecord)); @@ -267,10 +254,7 @@ public class ShardConsumerSubscriberTest { return null; }).when(shardConsumer).handleInput(any(ProcessRecordsInput.class), any(Subscription.class)); - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - processedNotifier.wait(5000); - } + startSubscriptionsAndWait(); synchronized (processedNotifier) { executorService.execute(() -> { @@ -290,7 +274,7 @@ public class ShardConsumerSubscriberTest { // // Wait for our blocking thread to control the thread in the executor. // - processedNotifier.wait(5000); + processedNotifier.wait(DEFAULT_NOTIFIER_TIMEOUT); } Stream.iterate(2, i -> i + 1).limit(97).forEach(this::addUniqueItem); @@ -301,7 +285,7 @@ public class ShardConsumerSubscriberTest { assertThat(subscriber.healthCheck(1), nullValue()); barrier.await(500, TimeUnit.MILLISECONDS); - processedNotifier.wait(5000); + processedNotifier.wait(DEFAULT_NOTIFIER_TIMEOUT); } verify(shardConsumer, times(100)).handleInput(argThat(eqProcessRecordsInput(processRecordsInput)), any(Subscription.class)); @@ -337,9 +321,7 @@ public class ShardConsumerSubscriberTest { }).when(shardConsumer).handleInput(any(ProcessRecordsInput.class), any(Subscription.class)); // First try to start subscriptions. - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - } + startSubscriptionsAndWait(100); // Verifying that there are no interactions with shardConsumer mock indicating no records were sent back and // subscription has not started correctly. @@ -397,9 +379,7 @@ public class ShardConsumerSubscriberTest { }).when(shardConsumer).handleInput(any(ProcessRecordsInput.class), any(Subscription.class)); // First try to start subscriptions. - synchronized (processedNotifier) { - subscriber.startSubscriptions(); - } + startSubscriptionsAndWait(100); // Verifying that there are no interactions with shardConsumer mock indicating no records were sent back and // subscription has not started correctly. @@ -478,6 +458,17 @@ public class ShardConsumerSubscriberTest { }).when(shardConsumer).handleInput(any(ProcessRecordsInput.class), any(Subscription.class)); } + private void startSubscriptionsAndWait() throws InterruptedException { + startSubscriptionsAndWait(DEFAULT_NOTIFIER_TIMEOUT); + } + + private void startSubscriptionsAndWait(long timeout) throws InterruptedException { + synchronized (processedNotifier) { + subscriber.startSubscriptions(); + processedNotifier.wait(timeout); + } + } + private class ResponseItem { private final RecordsRetrieved recordsRetrieved; private final Throwable throwable; From 88246e717e82dd9d877d3287de445b7fc73b06c4 Mon Sep 17 00:00:00 2001 From: Eric Meisel Date: Sun, 26 Mar 2023 08:24:22 -0500 Subject: [PATCH 48/88] Add simple SingleStreamTracker constructor with position (#1086) --- .../amazon/kinesis/processor/SingleStreamTracker.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java index 703c4881..fd1e045c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java @@ -58,6 +58,12 @@ public class SingleStreamTracker implements StreamTracker { this(streamIdentifier, new StreamConfig(streamIdentifier, initialPosition)); } + public SingleStreamTracker( + String streamName, + @NonNull InitialPositionInStreamExtended initialPosition) { + this(StreamIdentifier.singleStreamInstance(streamName), initialPosition); + } + public SingleStreamTracker(@NonNull StreamIdentifier streamIdentifier, @NonNull StreamConfig streamConfig) { this.streamIdentifier = streamIdentifier; this.streamConfigs = Collections.singletonList(streamConfig); From 7cd7c27a803b73b79d57e42590b1bbb87d756e7e Mon Sep 17 00:00:00 2001 From: Yu Zeng Date: Mon, 10 Apr 2023 16:27:11 -0700 Subject: [PATCH 49/88] Bump awssdk.version from 2.20.8 to 2.20.40 (#1089) Co-authored-by: Yu Zeng --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 87ece5e2..5c41c20e 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.20.8 + 2.20.40 From 7b23ae9b3c0af40dea8d4fe2fa2186e31c2d5f62 Mon Sep 17 00:00:00 2001 From: chenylee-aws <122478603+chenylee-aws@users.noreply.github.com> Date: Wed, 12 Apr 2023 10:33:50 -0700 Subject: [PATCH 50/88] Minimize race in PSSM to avoid unnecessary shard sync calls (#1088) --- .../coordinator/PeriodicShardSyncManager.java | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java index a885c4d9..dac77351 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java @@ -50,6 +50,7 @@ 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.Optional; @@ -178,23 +179,34 @@ class PeriodicShardSyncManager { final long runStartMillis = System.currentTimeMillis(); try { + // Create a copy of the streams to be considered for this run to avoid data race with Scheduler. + final Set streamConfigMap = new HashSet<>(currentStreamConfigMap.keySet()); + // Construct the stream to leases map to be used in the lease sync - final Map> streamToLeasesMap = getStreamToLeasesMap( - currentStreamConfigMap.keySet()); + final Map> streamToLeasesMap = getStreamToLeasesMap(streamConfigMap); // For each of the stream, check if shard sync needs to be done based on the leases state. - for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { - final ShardSyncResponse shardSyncResponse = checkForShardSync(streamConfigEntry.getKey(), - streamToLeasesMap.get(streamConfigEntry.getKey())); + for (StreamIdentifier streamIdentifier : streamConfigMap) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + log.info("Skipping shard sync task for {} as stream is purged", streamIdentifier); + continue; + } + final ShardSyncResponse shardSyncResponse = checkForShardSync(streamIdentifier, + streamToLeasesMap.get(streamIdentifier)); numStreamsWithPartialLeases += shardSyncResponse.isHoleDetected() ? 1 : 0; numStreamsToSync += shardSyncResponse.shouldDoShardSync ? 1 : 0; if (shardSyncResponse.shouldDoShardSync()) { log.info("Periodic shard syncer initiating shard sync for {} due to the reason - {} ", - streamConfigEntry.getKey(), shardSyncResponse.reasonForDecision()); + streamIdentifier, shardSyncResponse.reasonForDecision()); + final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); + if (streamConfig == null) { + log.info("Skipping shard sync task for {} as stream is purged", streamIdentifier); + continue; + } final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider - .apply(streamConfigEntry.getValue()); + .apply(streamConfig); if (!shardSyncTaskManager.submitShardSyncTask()) { log.warn( "Failed to submit shard sync task for stream {}. This could be due to the previous pending shard sync task.", @@ -205,7 +217,7 @@ class PeriodicShardSyncManager { shardSyncResponse.reasonForDecision()); } } else { - log.info("Skipping shard sync for {} due to the reason - {}", streamConfigEntry.getKey(), + log.info("Skipping shard sync for {} due to the reason - {}", streamIdentifier, shardSyncResponse.reasonForDecision()); } } From 5e7d4788ecd6aaa7dda634ebd3169a605b7032eb Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Tue, 18 Apr 2023 14:58:27 -0400 Subject: [PATCH 51/88] Code cleanup to introduce better testing and simplify future removal of (#1094) deprecated parameters (e.g., `Either appStreamTracker`). --- .../amazon/kinesis/common/StreamConfig.java | 2 + .../kinesis/processor/ProcessorConfig.java | 6 +- .../kinesis/retrieval/RetrievalConfig.java | 58 +++--- .../retrieval/RetrievalSpecificConfig.java | 22 ++- .../retrieval/fanout/FanOutConfig.java | 13 +- .../retrieval/polling/PollingConfig.java | 10 + .../kinesis/common/ConfigsBuilderTest.java | 11 +- .../kinesis/common/StreamConfigTest.java | 14 ++ .../kinesis/lifecycle/ShardConsumerTest.java | 119 ++++++------ .../retrieval/RetrievalConfigTest.java | 39 +++- .../retrieval/fanout/FanOutConfigTest.java | 182 ++++++++++-------- .../retrieval/polling/PollingConfigTest.java | 47 +++++ 12 files changed, 324 insertions(+), 199 deletions(-) create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamConfigTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PollingConfigTest.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java index b1057f13..8ca75dec 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java @@ -16,11 +16,13 @@ package software.amazon.kinesis.common; import lombok.Data; +import lombok.NonNull; import lombok.experimental.Accessors; @Data @Accessors(fluent = true) public class StreamConfig { + @NonNull private final StreamIdentifier streamIdentifier; private final InitialPositionInStreamExtended initialPositionInStreamExtended; private String consumerArn; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ProcessorConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ProcessorConfig.java index 04ea6614..7641bc44 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ProcessorConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ProcessorConfig.java @@ -15,9 +15,9 @@ package software.amazon.kinesis.processor; - import lombok.Data; - import lombok.NonNull; - import lombok.experimental.Accessors; +import lombok.Data; +import lombok.NonNull; +import lombok.experimental.Accessors; /** * Used by the KCL to configure the processor for processing the records. diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 3f001057..8ada4970 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -133,6 +133,8 @@ public class RetrievalConfig { } /** + * Convenience method to reconfigure the embedded {@link StreamTracker}, + * but only when not in multi-stream mode. * * @param initialPositionInStreamExtended * @@ -142,62 +144,46 @@ public class RetrievalConfig { */ @Deprecated public RetrievalConfig initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) { - this.appStreamTracker.apply(multiStreamTracker -> { + if (streamTracker().isMultiStream()) { throw new IllegalArgumentException( "Cannot set initialPositionInStreamExtended when multiStreamTracker is set"); - }, sc -> { - final StreamConfig updatedConfig = new StreamConfig(sc.streamIdentifier(), initialPositionInStreamExtended); - streamTracker = new SingleStreamTracker(sc.streamIdentifier(), updatedConfig); - appStreamTracker = Either.right(updatedConfig); - }); + }; + + final StreamIdentifier streamIdentifier = getSingleStreamIdentifier(); + final StreamConfig updatedConfig = new StreamConfig(streamIdentifier, initialPositionInStreamExtended); + streamTracker = new SingleStreamTracker(streamIdentifier, updatedConfig); + appStreamTracker = Either.right(updatedConfig); return this; } public RetrievalConfig retrievalSpecificConfig(RetrievalSpecificConfig retrievalSpecificConfig) { + retrievalSpecificConfig.validateState(streamTracker.isMultiStream()); this.retrievalSpecificConfig = retrievalSpecificConfig; - validateFanoutConfig(); - validatePollingConfig(); return this; } public RetrievalFactory retrievalFactory() { if (retrievalFactory == null) { if (retrievalSpecificConfig == null) { - retrievalSpecificConfig = new FanOutConfig(kinesisClient()) + final FanOutConfig fanOutConfig = new FanOutConfig(kinesisClient()) .applicationName(applicationName()); - retrievalSpecificConfig = appStreamTracker.map(multiStreamTracker -> retrievalSpecificConfig, - streamConfig -> ((FanOutConfig) retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName())); + if (!streamTracker.isMultiStream()) { + final String streamName = getSingleStreamIdentifier().streamName(); + fanOutConfig.streamName(streamName); + } + retrievalSpecificConfig(fanOutConfig); } retrievalFactory = retrievalSpecificConfig.retrievalFactory(); } return retrievalFactory; } - private void validateFanoutConfig() { - // If we are in multistream mode and if retrievalSpecificConfig is an instance of FanOutConfig and if consumerArn is set throw exception. - boolean isFanoutConfig = retrievalSpecificConfig instanceof FanOutConfig; - boolean isInvalidFanoutConfig = isFanoutConfig && appStreamTracker.map( - multiStreamTracker -> ((FanOutConfig) retrievalSpecificConfig).consumerArn() != null - || ((FanOutConfig) retrievalSpecificConfig).streamName() != null, - streamConfig -> streamConfig.streamIdentifier() == null - || streamConfig.streamIdentifier().streamName() == null); - if(isInvalidFanoutConfig) { - throw new IllegalArgumentException( - "Invalid config: Either in multi-stream mode with streamName/consumerArn configured or in single-stream mode with no streamName configured"); - } + /** + * Convenience method to return the {@link StreamIdentifier} from a + * single-stream tracker. + */ + private StreamIdentifier getSingleStreamIdentifier() { + return streamTracker.streamConfigList().get(0).streamIdentifier(); } - private void validatePollingConfig() { - boolean isPollingConfig = retrievalSpecificConfig instanceof PollingConfig; - boolean isInvalidPollingConfig = isPollingConfig && appStreamTracker.map( - multiStreamTracker -> - ((PollingConfig) retrievalSpecificConfig).streamName() != null, - streamConfig -> - streamConfig.streamIdentifier() == null || streamConfig.streamIdentifier().streamName() == null); - - if (isInvalidPollingConfig) { - throw new IllegalArgumentException( - "Invalid config: Either in multi-stream mode with streamName configured or in single-stream mode with no streamName configured"); - } - } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java index 30562994..d38fe054 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java @@ -15,9 +15,6 @@ package software.amazon.kinesis.retrieval; -import java.util.function.Function; -import software.amazon.kinesis.retrieval.polling.DataFetcher; - public interface RetrievalSpecificConfig { /** * Creates and returns a retrieval factory for the specific configuration @@ -25,4 +22,23 @@ public interface RetrievalSpecificConfig { * @return a retrieval factory that can create an appropriate retriever */ RetrievalFactory retrievalFactory(); + + /** + * Validates this instance is configured properly. For example, this + * method may validate that the stream name, if one is required, is + * non-null. + *

+ * If not in a valid state, an informative unchecked Exception -- for + * example, an {@link IllegalArgumentException} -- should be thrown so + * the caller may rectify the misconfiguration. + * + * @param isMultiStream whether state should be validated for multi-stream + * + * @deprecated remove keyword `default` to force implementation-specific behavior + */ + @Deprecated + default void validateState(boolean isMultiStream) { + // TODO convert this to a non-default implementation in a "major" release + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java index 9318b996..16307377 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java @@ -80,10 +80,21 @@ public class FanOutConfig implements RetrievalSpecificConfig { */ private long retryBackoffMillis = 1000; - @Override public RetrievalFactory retrievalFactory() { + @Override + public RetrievalFactory retrievalFactory() { return new FanOutRetrievalFactory(kinesisClient, streamName, consumerArn, this::getOrCreateConsumerArn); } + @Override + public void validateState(final boolean isMultiStream) { + if (isMultiStream) { + if ((streamName() != null) || (consumerArn() != null)) { + throw new IllegalArgumentException( + "FanOutConfig must not have streamName/consumerArn configured in multi-stream mode"); + } + } + } + private String getOrCreateConsumerArn(String streamName) { FanOutConsumerRegistration registration = createConsumerRegistration(streamName); try { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java index a37e7121..4dd64016 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java @@ -143,4 +143,14 @@ public class PollingConfig implements RetrievalSpecificConfig { return new SynchronousBlockingRetrievalFactory(streamName(), kinesisClient(), recordsFetcherFactory, maxRecords(), kinesisRequestTimeout, dataFetcherProvider); } + + @Override + public void validateState(final boolean isMultiStream) { + if (isMultiStream) { + if (streamName() != null) { + throw new IllegalArgumentException( + "PollingConfig must not have streamName configured in multi-stream mode"); + } + } + } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java index 8ea8f818..87caaa34 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java @@ -22,10 +22,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.mockito.Mockito.mock; -import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.MockitoAnnotations; +import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -34,6 +34,7 @@ import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; +@RunWith(MockitoJUnitRunner.class) public class ConfigsBuilderTest { @Mock @@ -51,11 +52,6 @@ public class ConfigsBuilderTest { private static final String APPLICATION_NAME = ConfigsBuilderTest.class.getSimpleName(); private static final String WORKER_IDENTIFIER = "worker-id"; - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - } - @Test public void testTrackerConstruction() { final String streamName = "single-stream"; @@ -77,6 +73,7 @@ public class ConfigsBuilderTest { } private ConfigsBuilder createConfig(String streamName) { + // intentional invocation of constructor where streamName is a String return new ConfigsBuilder(streamName, APPLICATION_NAME, mockKinesisClient, mockDynamoClient, mockCloudWatchClient, WORKER_IDENTIFIER, mockShardProcessorFactory); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamConfigTest.java new file mode 100644 index 00000000..9ba3267d --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamConfigTest.java @@ -0,0 +1,14 @@ +package software.amazon.kinesis.common; + +import static software.amazon.kinesis.common.InitialPositionInStream.TRIM_HORIZON; + +import org.junit.Test; + +public class StreamConfigTest { + + @Test(expected = NullPointerException.class) + public void testNullStreamIdentifier() { + new StreamConfig(null, InitialPositionInStreamExtended.newInitialPosition(TRIM_HORIZON)); + } + +} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java index 46677fb9..62fd13ef 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerTest.java @@ -15,13 +15,15 @@ package software.amazon.kinesis.lifecycle; -import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; @@ -167,7 +169,7 @@ public class ShardConsumerTest { @After public void after() { List remainder = executorService.shutdownNow(); - assertThat(remainder.isEmpty(), equalTo(true)); + assertTrue(remainder.isEmpty()); } private class TestPublisher implements RecordsPublisher { @@ -267,8 +269,7 @@ public class ShardConsumerTest { mockSuccessfulShutdown(null); TestPublisher cache = new TestPublisher(); - ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, logWarningForTaskAfterMillis, - shardConsumerArgument, initialState, Function.identity(), 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(cache); boolean initComplete = false; while (!initComplete) { @@ -321,8 +322,7 @@ public class ShardConsumerTest { mockSuccessfulShutdown(null); TestPublisher cache = new TestPublisher(); - ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, logWarningForTaskAfterMillis, - shardConsumerArgument, initialState, Function.identity(), 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(cache); boolean initComplete = false; while (!initComplete) { @@ -341,7 +341,7 @@ public class ShardConsumerTest { // This will block if a lock is held on ShardConsumer#this // consumer.executeLifecycle(); - assertThat(consumer.isShutdown(), equalTo(false)); + assertFalse(consumer.isShutdown()); log.debug("Release processing task interlock"); awaitAndResetBarrier(processingTaskInterlock); @@ -370,7 +370,6 @@ public class ShardConsumerTest { @Test public void testDataArrivesAfterProcessing2() throws Exception { - CyclicBarrier taskCallBarrier = new CyclicBarrier(2); mockSuccessfulInitialize(null); @@ -380,8 +379,7 @@ public class ShardConsumerTest { mockSuccessfulShutdown(null); TestPublisher cache = new TestPublisher(); - ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, logWarningForTaskAfterMillis, - shardConsumerArgument, initialState, Function.identity(), 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(cache); boolean initComplete = false; while (!initComplete) { @@ -435,13 +433,10 @@ public class ShardConsumerTest { verifyNoMoreInteractions(taskExecutionListener); } - @SuppressWarnings("unchecked") @Test @Ignore public final void testInitializationStateUponFailure() throws Exception { - ShardConsumer consumer = new ShardConsumer(recordsPublisher, executorService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, initialState, Function.identity(), 1, - taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(recordsPublisher); when(initialState.createTask(eq(shardConsumerArgument), eq(consumer), any())).thenReturn(initializeTask); when(initializeTask.call()).thenReturn(new TaskResult(new Exception("Bad"))); @@ -468,17 +463,14 @@ public class ShardConsumerTest { /** * Test method to verify consumer undergoes the transition WAITING_ON_PARENT_SHARDS -> INITIALIZING -> PROCESSING */ - @SuppressWarnings("unchecked") @Test - public final void testSuccessfulConsumerStateTransition() throws Exception { + public final void testSuccessfulConsumerStateTransition() { ExecutorService directExecutorService = spy(executorService); - doAnswer(invocation -> directlyExecuteRunnable(invocation)) + doAnswer(this::directlyExecuteRunnable) .when(directExecutorService).execute(any()); - ShardConsumer consumer = new ShardConsumer(recordsPublisher, directExecutorService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, blockedOnParentsState, - t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(directExecutorService, blockedOnParentsState); mockSuccessfulUnblockOnParents(); mockSuccessfulInitializeWithFailureTransition(); @@ -502,20 +494,17 @@ public class ShardConsumerTest { * Test method to verify consumer does not transition to PROCESSING from WAITING_ON_PARENT_SHARDS when * INITIALIZING tasks gets rejected. */ - @SuppressWarnings("unchecked") @Test public final void testConsumerNotTransitionsToProcessingWhenInitializationFails() { ExecutorService failingService = spy(executorService); - ShardConsumer consumer = new ShardConsumer(recordsPublisher, failingService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, blockedOnParentsState, - t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(failingService, blockedOnParentsState); mockSuccessfulUnblockOnParents(); mockSuccessfulInitializeWithFailureTransition(); mockSuccessfulProcessing(null); // Failing the initialization task and all other attempts after that. - doAnswer(invocation -> directlyExecuteRunnable(invocation)) + doAnswer(this::directlyExecuteRunnable) .doThrow(new RejectedExecutionException()) .when(failingService).execute(any()); @@ -537,24 +526,21 @@ public class ShardConsumerTest { * Test method to verify consumer transition to PROCESSING from WAITING_ON_PARENT_SHARDS with * intermittent INITIALIZING task rejections. */ - @SuppressWarnings("unchecked") @Test public final void testConsumerTransitionsToProcessingWithIntermittentInitializationFailures() { ExecutorService failingService = spy(executorService); - ShardConsumer consumer = new ShardConsumer(recordsPublisher, failingService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, blockedOnParentsState, - t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(failingService, blockedOnParentsState); mockSuccessfulUnblockOnParents(); mockSuccessfulInitializeWithFailureTransition(); mockSuccessfulProcessing(null); // Failing the initialization task and few other attempts after that. - doAnswer(invocation -> directlyExecuteRunnable(invocation)) + doAnswer(this::directlyExecuteRunnable) .doThrow(new RejectedExecutionException()) .doThrow(new RejectedExecutionException()) .doThrow(new RejectedExecutionException()) - .doAnswer(invocation -> directlyExecuteRunnable(invocation)) + .doAnswer(this::directlyExecuteRunnable) .when(failingService).execute(any()); int arbitraryExecutionCount = 6; @@ -574,13 +560,10 @@ public class ShardConsumerTest { /** * Test method to verify consumer does not transition to INITIALIZING when WAITING_ON_PARENT_SHARDS task rejected. */ - @SuppressWarnings("unchecked") @Test public final void testConsumerNotTransitionsToInitializingWhenWaitingOnParentsFails() { ExecutorService failingService = spy(executorService); - ShardConsumer consumer = new ShardConsumer(recordsPublisher, failingService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, blockedOnParentsState, - t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(failingService, blockedOnParentsState); mockSuccessfulUnblockOnParentsWithFailureTransition(); mockSuccessfulInitializeWithFailureTransition(); @@ -606,13 +589,10 @@ public class ShardConsumerTest { /** * Test method to verify consumer stays in INITIALIZING state when InitializationTask fails. */ - @SuppressWarnings("unchecked") @Test(expected = RejectedExecutionException.class) public final void testInitializationStateUponSubmissionFailure() throws Exception { - ExecutorService failingService = mock(ExecutorService.class); - ShardConsumer consumer = new ShardConsumer(recordsPublisher, failingService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, initialState, t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(failingService, initialState); doThrow(new RejectedExecutionException()).when(failingService).execute(any()); @@ -625,8 +605,7 @@ public class ShardConsumerTest { @Test public void testErrorThrowableInInitialization() throws Exception { - ShardConsumer consumer = new ShardConsumer(recordsPublisher, executorService, shardInfo, - logWarningForTaskAfterMillis, shardConsumerArgument, initialState, t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(recordsPublisher); when(initialState.createTask(any(), any(), any())).thenReturn(initializeTask); when(initialState.taskType()).thenReturn(TaskType.INITIALIZE); @@ -645,12 +624,10 @@ public class ShardConsumerTest { @Test public void testRequestedShutdownWhileQuiet() throws Exception { - CyclicBarrier taskBarrier = new CyclicBarrier(2); TestPublisher cache = new TestPublisher(); - ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, logWarningForTaskAfterMillis, - shardConsumerArgument, initialState, t -> t, 1, taskExecutionListener, 0); + final ShardConsumer consumer = createShardConsumer(cache); mockSuccessfulInitialize(null); @@ -692,15 +669,15 @@ public class ShardConsumerTest { consumer.gracefulShutdown(shutdownNotification); boolean shutdownComplete = consumer.shutdownComplete().get(); - assertThat(shutdownComplete, equalTo(false)); + assertFalse(shutdownComplete); shutdownComplete = consumer.shutdownComplete().get(); - assertThat(shutdownComplete, equalTo(false)); + assertFalse(shutdownComplete); consumer.leaseLost(); shutdownComplete = consumer.shutdownComplete().get(); - assertThat(shutdownComplete, equalTo(false)); + assertFalse(shutdownComplete); shutdownComplete = consumer.shutdownComplete().get(); - assertThat(shutdownComplete, equalTo(true)); + assertTrue(shutdownComplete); verify(processingState, times(2)).createTask(any(), any(), any()); verify(shutdownRequestedState, never()).shutdownTransition(eq(ShutdownReason.LEASE_LOST)); @@ -776,7 +753,6 @@ public class ShardConsumerTest { @Test public void testLongRunningTasks() throws Exception { - TestPublisher cache = new TestPublisher(); ShardConsumer consumer = new ShardConsumer(cache, executorService, shardInfo, Optional.of(1L), @@ -792,19 +768,19 @@ public class ShardConsumerTest { CompletableFuture initSuccess = consumer.initializeComplete(); awaitAndResetBarrier(taskArriveBarrier); - assertThat(consumer.taskRunningTime(), notNullValue()); + assertNotNull(consumer.taskRunningTime()); consumer.healthCheck(); awaitAndResetBarrier(taskDepartBarrier); - assertThat(initSuccess.get(), equalTo(false)); + assertFalse(initSuccess.get()); verify(initializeTask).call(); initSuccess = consumer.initializeComplete(); verify(initializeTask).call(); - assertThat(initSuccess.get(), equalTo(true)); + assertTrue(initSuccess.get()); consumer.healthCheck(); - assertThat(consumer.taskRunningTime(), nullValue()); + assertNull(consumer.taskRunningTime()); consumer.subscribe(); cache.awaitInitialSetup(); @@ -813,14 +789,14 @@ public class ShardConsumerTest { awaitAndResetBarrier(taskArriveBarrier); Instant previousTaskStartTime = consumer.taskDispatchedAt(); - assertThat(consumer.taskRunningTime(), notNullValue()); + assertNotNull(consumer.taskRunningTime()); consumer.healthCheck(); awaitAndResetBarrier(taskDepartBarrier); consumer.healthCheck(); cache.requestBarrier.await(); - assertThat(consumer.taskRunningTime(), nullValue()); + assertNull(consumer.taskRunningTime()); cache.requestBarrier.reset(); // Sleep for 10 millis before processing next task. If we don't; then the following @@ -831,28 +807,28 @@ public class ShardConsumerTest { awaitAndResetBarrier(taskArriveBarrier); Instant currentTaskStartTime = consumer.taskDispatchedAt(); - assertThat(currentTaskStartTime, not(equalTo(previousTaskStartTime))); + assertNotEquals(currentTaskStartTime, previousTaskStartTime); awaitAndResetBarrier(taskDepartBarrier); cache.requestBarrier.await(); - assertThat(consumer.taskRunningTime(), nullValue()); + assertNull(consumer.taskRunningTime()); cache.requestBarrier.reset(); consumer.leaseLost(); - assertThat(consumer.isShutdownRequested(), equalTo(true)); + assertTrue(consumer.isShutdownRequested()); CompletableFuture shutdownComplete = consumer.shutdownComplete(); awaitAndResetBarrier(taskArriveBarrier); - assertThat(consumer.taskRunningTime(), notNullValue()); + assertNotNull(consumer.taskRunningTime()); awaitAndResetBarrier(taskDepartBarrier); - assertThat(shutdownComplete.get(), equalTo(false)); + assertFalse(shutdownComplete.get()); shutdownComplete = consumer.shutdownComplete(); - assertThat(shutdownComplete.get(), equalTo(true)); + assertTrue(shutdownComplete.get()); - assertThat(consumer.taskRunningTime(), nullValue()); + assertNull(consumer.taskRunningTime()); consumer.healthCheck(); verify(taskExecutionListener, times(1)).beforeTaskExecution(initialTaskInput); @@ -918,7 +894,6 @@ public class ShardConsumerTest { } private void mockSuccessfulInitialize(CyclicBarrier taskCallBarrier, CyclicBarrier taskInterlockBarrier) { - when(initialState.createTask(eq(shardConsumerArgument), any(), any())).thenReturn(initializeTask); when(initialState.taskType()).thenReturn(TaskType.INITIALIZE); when(initializeTask.taskType()).thenReturn(TaskType.INITIALIZE); @@ -968,4 +943,18 @@ public class ShardConsumerTest { return null; } + private ShardConsumer createShardConsumer(final RecordsPublisher publisher) { + return createShardConsumer(publisher, executorService, initialState); + } + + private ShardConsumer createShardConsumer(final ExecutorService executorService, final ConsumerState state) { + return createShardConsumer(recordsPublisher, executorService, state); + } + + private ShardConsumer createShardConsumer(final RecordsPublisher publisher, + final ExecutorService executorService, final ConsumerState state) { + return new ShardConsumer(publisher, executorService, shardInfo, logWarningForTaskAfterMillis, + shardConsumerArgument, state, Function.identity(), 1, taskExecutionListener, 0); + } + } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java index 041ac71e..464459d5 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java @@ -5,14 +5,19 @@ import java.util.Optional; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static software.amazon.kinesis.common.InitialPositionInStream.LATEST; import static software.amazon.kinesis.common.InitialPositionInStream.TRIM_HORIZON; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.MockitoAnnotations; +import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; @@ -20,6 +25,7 @@ import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; +@RunWith(MockitoJUnitRunner.class) public class RetrievalConfigTest { private static final String APPLICATION_NAME = RetrievalConfigTest.class.getSimpleName(); @@ -27,9 +33,12 @@ public class RetrievalConfigTest { @Mock private KinesisAsyncClient mockKinesisClient; + @Mock + private MultiStreamTracker mockMultiStreamTracker; + @Before public void setUp() { - MockitoAnnotations.initMocks(this); + when(mockMultiStreamTracker.isMultiStream()).thenReturn(true); } @Test @@ -69,11 +78,33 @@ public class RetrievalConfigTest { @Test(expected = IllegalArgumentException.class) public void testUpdateInitialPositionInMultiStream() { - final RetrievalConfig config = createConfig(mock(MultiStreamTracker.class)); - config.initialPositionInStreamExtended( + createConfig(mockMultiStreamTracker).initialPositionInStreamExtended( InitialPositionInStreamExtended.newInitialPosition(TRIM_HORIZON)); } + /** + * Test that an invalid {@link RetrievalSpecificConfig} does not overwrite + * a valid one. + */ + @Test + public void testInvalidRetrievalSpecificConfig() { + final RetrievalSpecificConfig validConfig = mock(RetrievalSpecificConfig.class); + final RetrievalSpecificConfig invalidConfig = mock(RetrievalSpecificConfig.class); + doThrow(new IllegalArgumentException("womp womp")).when(invalidConfig).validateState(true); + + final RetrievalConfig config = createConfig(mockMultiStreamTracker); + assertNull(config.retrievalSpecificConfig()); + config.retrievalSpecificConfig(validConfig); + assertEquals(validConfig, config.retrievalSpecificConfig()); + + try { + config.retrievalSpecificConfig(invalidConfig); + Assert.fail("should throw"); + } catch (RuntimeException re) { + assertEquals(validConfig, config.retrievalSpecificConfig()); + } + } + private RetrievalConfig createConfig(String streamName) { return new RetrievalConfig(mockKinesisClient, streamName, APPLICATION_NAME); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java index 4fee3d08..32ca17ce 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java @@ -15,16 +15,20 @@ package software.amazon.kinesis.retrieval.fanout; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -55,132 +59,150 @@ public class FanOutConfigTest { @Mock private StreamConfig streamConfig; + private FanOutConfig config; + @Before public void setup() { - when(streamConfig.consumerArn()).thenReturn(null); + config = spy(new FanOutConfig(kinesisClient)) + // DRY: set the most commonly-used parameters + .applicationName(TEST_APPLICATION_NAME) + .streamName(TEST_STREAM_NAME); + doReturn(consumerRegistration).when(config) + .createConsumerRegistration(eq(kinesisClient), anyString(), anyString()); } @Test - public void testNoRegisterIfConsumerArnSet() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).consumerArn(TEST_CONSUMER_ARN); + public void testNoRegisterIfConsumerArnSet() { + config.consumerArn(TEST_CONSUMER_ARN) + // unset common parameters + .applicationName(null).streamName(null); + RetrievalFactory retrievalFactory = config.retrievalFactory(); - assertThat(retrievalFactory, not(nullValue())); - verify(consumerRegistration, never()).getOrCreateStreamConsumerArn(); + assertNotNull(retrievalFactory); + verifyZeroInteractions(consumerRegistration); } @Test public void testRegisterCalledWhenConsumerArnUnset() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) - .streamName(TEST_STREAM_NAME); - RetrievalFactory retrievalFactory = config.retrievalFactory(); - ShardInfo shardInfo = mock(ShardInfo.class); -// doReturn(Optional.of(StreamIdentifier.singleStreamInstance(TEST_STREAM_NAME).serialize())).when(shardInfo).streamIdentifier(); - doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); - assertThat(retrievalFactory, not(nullValue())); + getRecordsCache(null); + verify(consumerRegistration).getOrCreateStreamConsumerArn(); } @Test public void testRegisterNotCalledWhenConsumerArnSetInMultiStreamMode() throws Exception { when(streamConfig.consumerArn()).thenReturn("consumerArn"); - FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) - .streamName(TEST_STREAM_NAME); - RetrievalFactory retrievalFactory = config.retrievalFactory(); - ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.of("account:stream:12345")).when(shardInfo).streamIdentifierSerOpt(); - retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); - assertThat(retrievalFactory, not(nullValue())); + + getRecordsCache("account:stream:12345"); + verify(consumerRegistration, never()).getOrCreateStreamConsumerArn(); } @Test public void testRegisterCalledWhenConsumerArnNotSetInMultiStreamMode() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) - .streamName(TEST_STREAM_NAME); - RetrievalFactory retrievalFactory = config.retrievalFactory(); - ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.of("account:stream:12345")).when(shardInfo).streamIdentifierSerOpt(); - retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); - assertThat(retrievalFactory, not(nullValue())); + getRecordsCache("account:stream:12345"); + verify(consumerRegistration).getOrCreateStreamConsumerArn(); } @Test public void testDependencyExceptionInConsumerCreation() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) - .streamName(TEST_STREAM_NAME); DependencyException de = new DependencyException("Bad", null); when(consumerRegistration.getOrCreateStreamConsumerArn()).thenThrow(de); + try { - config.retrievalFactory(); + getRecordsCache(null); + Assert.fail("should throw"); } catch (RuntimeException e) { verify(consumerRegistration).getOrCreateStreamConsumerArn(); - assertThat(e.getCause(), equalTo(de)); + assertEquals(de, e.getCause()); } } @Test - public void testCreationWithApplicationName() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) - .streamName(TEST_STREAM_NAME); - RetrievalFactory factory = config.retrievalFactory(); - ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); - assertThat(factory, not(nullValue())); + public void testCreationWithApplicationName() { + getRecordsCache(null); - TestingConfig testingConfig = (TestingConfig) config; - assertThat(testingConfig.stream, equalTo(TEST_STREAM_NAME)); - assertThat(testingConfig.consumerToCreate, equalTo(TEST_APPLICATION_NAME)); + assertEquals(TEST_STREAM_NAME, config.streamName()); + assertEquals(TEST_APPLICATION_NAME, config.applicationName()); } @Test - public void testCreationWithConsumerName() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).consumerName(TEST_CONSUMER_NAME) - .streamName(TEST_STREAM_NAME); - RetrievalFactory factory = config.retrievalFactory(); - ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); - assertThat(factory, not(nullValue())); - TestingConfig testingConfig = (TestingConfig) config; - assertThat(testingConfig.stream, equalTo(TEST_STREAM_NAME)); - assertThat(testingConfig.consumerToCreate, equalTo(TEST_CONSUMER_NAME)); + public void testCreationWithConsumerName() { + config.consumerName(TEST_CONSUMER_NAME) + // unset common parameters + .applicationName(null); + + getRecordsCache(null); + + assertEquals(TEST_STREAM_NAME, config.streamName()); + assertEquals(TEST_CONSUMER_NAME, config.consumerName()); } @Test - public void testCreationWithBothConsumerApplication() throws Exception { - FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) - .consumerName(TEST_CONSUMER_NAME).streamName(TEST_STREAM_NAME); - RetrievalFactory factory = config.retrievalFactory(); - ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); - assertThat(factory, not(nullValue())); + public void testCreationWithBothConsumerApplication() { + config = config.consumerName(TEST_CONSUMER_NAME); - TestingConfig testingConfig = (TestingConfig) config; - assertThat(testingConfig.stream, equalTo(TEST_STREAM_NAME)); - assertThat(testingConfig.consumerToCreate, equalTo(TEST_CONSUMER_NAME)); + getRecordsCache(null); + + assertEquals(TEST_STREAM_NAME, config.streamName()); + assertEquals(TEST_CONSUMER_NAME, config.consumerName()); } - private class TestingConfig extends FanOutConfig { + @Test + public void testValidState() { + assertNull(config.consumerArn()); + assertNotNull(config.streamName()); - String stream; - String consumerToCreate; + config.validateState(false); - public TestingConfig(KinesisAsyncClient kinesisClient) { - super(kinesisClient); + // both streamName and consumerArn are non-null + config.consumerArn(TEST_CONSUMER_ARN); + config.validateState(false); + + config.consumerArn(null); + config.streamName(null); + config.validateState(false); + config.validateState(true); + + assertNull(config.streamName()); + assertNull(config.consumerArn()); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidStateMultiWithStreamName() { + testInvalidState(TEST_STREAM_NAME, null); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidStateMultiWithConsumerArn() { + testInvalidState(null, TEST_CONSUMER_ARN); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidStateMultiWithStreamNameAndConsumerArn() { + testInvalidState(TEST_STREAM_NAME, TEST_CONSUMER_ARN); + } + + private void testInvalidState(final String streamName, final String consumerArn) { + config.streamName(streamName); + config.consumerArn(consumerArn); + + try { + config.validateState(true); + } finally { + assertEquals(streamName, config.streamName()); + assertEquals(consumerArn, config.consumerArn()); } + } - @Override - protected FanOutConsumerRegistration createConsumerRegistration(KinesisAsyncClient client, String stream, - String consumerToCreate) { - this.stream = stream; - this.consumerToCreate = consumerToCreate; - return consumerRegistration; - } + private void getRecordsCache(final String streamIdentifer) { + final ShardInfo shardInfo = mock(ShardInfo.class); + when(shardInfo.streamIdentifierSerOpt()).thenReturn(Optional.ofNullable(streamIdentifer)); + + final RetrievalFactory factory = config.retrievalFactory(); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); } } \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PollingConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PollingConfigTest.java new file mode 100644 index 00000000..760c6dce --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PollingConfigTest.java @@ -0,0 +1,47 @@ +package software.amazon.kinesis.retrieval.polling; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; + +@RunWith(MockitoJUnitRunner.class) +public class PollingConfigTest { + + private static final String STREAM_NAME = PollingConfigTest.class.getSimpleName(); + + @Mock + private KinesisAsyncClient mockKinesisClinet; + + private PollingConfig config; + + @Before + public void setUp() { + config = new PollingConfig(mockKinesisClinet); + } + + @Test + public void testValidState() { + assertNull(config.streamName()); + + config.validateState(true); + config.validateState(false); + + config.streamName(STREAM_NAME); + config.validateState(false); + assertEquals(STREAM_NAME, config.streamName()); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidStateMultiWithStreamName() { + config.streamName(STREAM_NAME); + + config.validateState(true); + } + +} \ No newline at end of file From fc52976c3d75f9199394bad8615ffbcb31bc93d2 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Tue, 18 Apr 2023 14:58:46 -0400 Subject: [PATCH 52/88] Added `SupplierCache` (so it may be leveraged by StreamARN work). (#1096) --- .../amazon/kinesis/common/SupplierCache.java | 36 ++++++++++++ .../kinesis/common/SupplierCacheTest.java | 56 +++++++++++++++++++ 2 files changed, 92 insertions(+) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java new file mode 100644 index 00000000..632e4b8f --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java @@ -0,0 +1,36 @@ +package software.amazon.kinesis.common; + +import java.util.function.Supplier; + +import lombok.RequiredArgsConstructor; + +/** + * Caches results from a {@link Supplier}. Caching is especially useful when + * {@link Supplier#get()} is an expensive call that produces static results. + */ +@RequiredArgsConstructor +public class SupplierCache { + + private final Supplier supplier; + + private volatile T result; + + /** + * Returns the cached result. If the cache is null, the supplier will be + * invoked to populate the cache. + * + * @return cached result which may be null + */ + public T get() { + if (result == null) { + synchronized (this) { + // double-check lock + if (result == null) { + result = supplier.get(); + } + } + } + return result; + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java new file mode 100644 index 00000000..4df4f81d --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java @@ -0,0 +1,56 @@ +package software.amazon.kinesis.common; + +import java.util.function.Supplier; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class SupplierCacheTest { + + private static final Object DUMMY_RESULT = SupplierCacheTest.class; + + @Mock + private Supplier mockSupplier; + + private SupplierCache cache; + + @Before + public void setUp() { + cache = new SupplierCache<>(mockSupplier); + } + + @Test + public void testCache() { + when(mockSupplier.get()).thenReturn(DUMMY_RESULT); + + final Object result1 = cache.get(); + final Object result2 = cache.get(); + + assertEquals(DUMMY_RESULT, result1); + assertSame(result1, result2); + verify(mockSupplier).get(); + } + + @Test + public void testCacheWithNullResult() { + when(mockSupplier.get()).thenReturn(null).thenReturn(DUMMY_RESULT); + + final Object result1 = cache.get(); + final Object result2 = cache.get(); + + assertNull(result1); + assertEquals(DUMMY_RESULT, result2); + verify(mockSupplier, times(2)).get(); + } +} \ No newline at end of file From 52e34dbe8f03eae4ae47435bfe3a687303666a16 Mon Sep 17 00:00:00 2001 From: Yu Zeng Date: Tue, 18 Apr 2023 12:26:31 -0700 Subject: [PATCH 53/88] Internally construct StreamARN using STS (#1087) Co-authored-by: Yu Zeng Co-authored-by: stair <123031771+stair-aws@users.noreply.github.com> --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 21 +++- .../amazon/kinesis/common/ConfigsBuilder.java | 2 +- .../amazon/kinesis/common/StreamARNUtil.java | 86 +++++++++++++ .../kinesis/common/StreamIdentifier.java | 76 ++++++++---- .../kinesis/leases/KinesisShardDetector.java | 16 +-- .../processor/SingleStreamTracker.java | 5 + .../kinesis/retrieval/RetrievalConfig.java | 6 +- .../retrieval/polling/KinesisDataFetcher.java | 11 +- .../kinesis/common/ConfigsBuilderTest.java | 4 +- .../kinesis/common/StreamARNUtilTest.java | 113 ++++++++++++++++++ .../kinesis/common/StreamIdentifierTest.java | 94 +++++++++++++++ .../kinesis/coordinator/SchedulerTest.java | 3 +- .../retrieval/RetrievalConfigTest.java | 3 +- .../kinesis/utils/MockObjectHelper.java | 31 +++++ pom.xml | 2 +- 16 files changed, 435 insertions(+), 40 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 6290ed1c..dd20862e 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.4.9-SNAPSHOT + 2.5.0-SNAPSHOT 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 3f17aa58..7356a72b 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.4.9-SNAPSHOT + 2.5.0-SNAPSHOT amazon-kinesis-client @@ -75,6 +75,11 @@ netty-nio-client ${awssdk.version} + + software.amazon.awssdk + sts + ${awssdk.version} + software.amazon.glue schema-registry-serde @@ -134,6 +139,20 @@ test + + org.powermock + powermock-module-junit4 + 1.7.4 + test + + + + org.powermock + powermock-api-mockito + 1.7.4 + test + + org.hamcrest hamcrest-all diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java index a5bbfebe..57de9059 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java @@ -142,7 +142,7 @@ public class ConfigsBuilder { @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { - this(new SingleStreamTracker(streamName), + this(new SingleStreamTracker(streamName, kinesisClient.serviceClientConfiguration().region()), applicationName, kinesisClient, dynamoDBClient, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java new file mode 100644 index 00000000..7fe2d435 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java @@ -0,0 +1,86 @@ +package software.amazon.kinesis.common; + +import com.google.common.base.Joiner; +import lombok.AccessLevel; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; + +import java.util.HashMap; +import java.util.Optional; + +@Slf4j +@NoArgsConstructor(access = AccessLevel.PRIVATE) +public final class StreamARNUtil { + private static final HashMap streamARNCache = new HashMap<>(); + + /** + * This static method attempts to retrieve the stream ARN using the stream name, region, and accountId returned by STS + * It is designed to fail gracefully, returning Optional.empty() if any errors occur. + * @param streamName: stream name + * @param kinesisRegion: kinesisRegion is a nullable parameter used to construct the stream arn + * @return + */ + public static Optional getStreamARN(String streamName, Region kinesisRegion) { + return getStreamARN(streamName, kinesisRegion, Optional.empty()); + } + + public static Optional getStreamARN(String streamName, Region kinesisRegion, @NonNull Optional accountId) { + if (kinesisRegion == null || StringUtils.isEmpty(kinesisRegion.toString())) { + return Optional.empty(); + } + // Consult the cache before contacting STS + String key = getCacheKey(streamName, kinesisRegion, accountId); + if (streamARNCache.containsKey(key)) { + return Optional.of(streamARNCache.get(key)); + } + + Optional stsCallerArn = getStsCallerArn(); + if (!stsCallerArn.isPresent() || !stsCallerArn.get().accountId().isPresent()) { + return Optional.empty(); + } + accountId = accountId.isPresent() ? accountId : stsCallerArn.get().accountId(); + Arn kinesisStreamArn = Arn.builder() + .partition(stsCallerArn.get().partition()) + .service("kinesis") + .region(kinesisRegion.toString()) + .accountId(accountId.get()) + .resource("stream/" + streamName) + .build(); + + // Update the cache + streamARNCache.put(key, kinesisStreamArn); + return Optional.of(kinesisStreamArn); + } + + private static Optional getStsCallerArn() { + GetCallerIdentityResponse response; + try { + response = getStsClient().getCallerIdentity(); + } catch (AwsServiceException | SdkClientException e) { + log.warn("Unable to get sts caller identity to build stream arn", e); + return Optional.empty(); + } + return Optional.of(Arn.fromString(response.arn())); + } + + private static StsClient getStsClient() { + return StsClient.builder() + .httpClient(UrlConnectionHttpClient.builder().build()) + .build(); + } + + private static String getCacheKey( + String streamName, @NonNull Region kinesisRegion, @NonNull Optional accountId) { + return Joiner.on(":").join(streamName, kinesisRegion.toString(), accountId.orElse("")); + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java index 1259a609..0fc7d2b2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -16,38 +16,39 @@ package software.amazon.kinesis.common; import com.google.common.base.Joiner; +import lombok.AccessLevel; +import lombok.Builder; import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.NonNull; +import lombok.ToString; import lombok.experimental.Accessors; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.utils.Validate; import java.util.Optional; import java.util.regex.Pattern; -@EqualsAndHashCode @Getter @Accessors(fluent = true) +@Builder(access = AccessLevel.PRIVATE) +@EqualsAndHashCode +@Getter +@ToString +@Accessors(fluent = true) public class StreamIdentifier { - private final Optional accountIdOptional; + @Builder.Default + private final Optional accountIdOptional = Optional.empty(); private final String streamName; - private final Optional streamCreationEpochOptional; + @Builder.Default + private final Optional streamCreationEpochOptional = Optional.empty(); + @Builder.Default + private final Optional streamARNOptional = Optional.empty(); private static final String DELIMITER = ":"; - private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*"); - - private StreamIdentifier(@NonNull String accountId, @NonNull String streamName, @NonNull Long streamCreationEpoch) { - this.accountIdOptional = Optional.of(accountId); - this.streamName = streamName; - this.streamCreationEpochOptional = Optional.of(streamCreationEpoch); - } - - private StreamIdentifier(@NonNull String streamName) { - this.accountIdOptional = Optional.empty(); - this.streamName = streamName; - this.streamCreationEpochOptional = Optional.empty(); - } + private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*" + ":?([a-z]{2}(-gov)?-[a-z]+-\\d{1})?"); /** * Serialize the current StreamIdentifier instance. + * TODO: Consider appending region info for cross-account consumer support * @return */ public String serialize() { @@ -63,14 +64,35 @@ public class StreamIdentifier { /** * Create a multi stream instance for StreamIdentifier from serialized stream identifier. - * The serialized stream identifier should be of the format account:stream:creationepoch + * See the format of a serialized stream identifier at {@link StreamIdentifier#multiStreamInstance(String, Region)} * @param streamIdentifierSer * @return StreamIdentifier */ public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) { + return multiStreamInstance(streamIdentifierSer, null); + } + + /** + * Create a multi stream instance for StreamIdentifier from serialized stream identifier. + * @param streamIdentifierSer The serialized stream identifier should be of the format + * account:stream:creationepoch[:region] + * @param kinesisRegion This nullable region is used to construct the optional StreamARN + * @return StreamIdentifier + */ + public static StreamIdentifier multiStreamInstance(String streamIdentifierSer, Region kinesisRegion) { if (PATTERN.matcher(streamIdentifierSer).matches()) { final String[] split = streamIdentifierSer.split(DELIMITER); - return new StreamIdentifier(split[0], split[1], Long.parseLong(split[2])); + final String streamName = split[1]; + final Optional accountId = Optional.ofNullable(split[0]); + StreamIdentifierBuilder builder = StreamIdentifier.builder() + .accountIdOptional(accountId) + .streamName(streamName) + .streamCreationEpochOptional(Optional.of(Long.parseLong(split[2]))); + final Region region = (split.length == 4) ? + Region.of(split[3]) : // Use the region extracted from the serialized string, which matches the regex pattern + kinesisRegion; // Otherwise just use the provided region + final Optional streamARN = StreamARNUtil.getStreamARN(streamName, region, accountId); + return builder.streamARNOptional(streamARN).build(); } else { throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer); } @@ -82,7 +104,21 @@ public class StreamIdentifier { * @return StreamIdentifier */ public static StreamIdentifier singleStreamInstance(String streamName) { + return singleStreamInstance(streamName, null); + } + + /** + * Create a single stream instance for StreamIdentifier from the provided stream name and kinesisRegion. + * This method also constructs the optional StreamARN based on the region info. + * @param streamName + * @param kinesisRegion + * @return StreamIdentifier + */ + public static StreamIdentifier singleStreamInstance(String streamName, Region kinesisRegion) { Validate.notEmpty(streamName, "StreamName should not be empty"); - return new StreamIdentifier(streamName); + return StreamIdentifier.builder() + .streamName(streamName) + .streamARNOptional(StreamARNUtil.getStreamARN(streamName, kinesisRegion)) + .build(); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java index 9a44a553..04b7c795 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -222,11 +222,12 @@ public class KinesisShardDetector implements ShardDetector { final boolean shouldPropagateResourceNotFoundException) { ListShardsRequest.Builder builder = KinesisRequestsBuilder.listShardsRequestBuilder(); if (StringUtils.isEmpty(nextToken)) { - builder = builder.streamName(streamIdentifier.streamName()).shardFilter(shardFilter); - } else { - builder = builder.nextToken(nextToken); - } + builder.streamName(streamIdentifier.streamName()).shardFilter(shardFilter); + streamIdentifier.streamARNOptional().ifPresent(arn -> builder.streamARN(arn.toString())); + } else { + builder.nextToken(nextToken); + } final ListShardsRequest request = builder.build(); log.info("Stream {}: listing shards with list shards request {}", streamIdentifier, request); @@ -308,11 +309,12 @@ public class KinesisShardDetector implements ShardDetector { @Override public List getChildShards(final String shardId) throws InterruptedException, ExecutionException, TimeoutException { - final GetShardIteratorRequest getShardIteratorRequest = KinesisRequestsBuilder.getShardIteratorRequestBuilder() + final GetShardIteratorRequest.Builder requestBuilder = KinesisRequestsBuilder.getShardIteratorRequestBuilder() .streamName(streamIdentifier.streamName()) .shardIteratorType(ShardIteratorType.LATEST) - .shardId(shardId) - .build(); + .shardId(shardId); + streamIdentifier.streamARNOptional().ifPresent(arn -> requestBuilder.streamARN(arn.toString())); + final GetShardIteratorRequest getShardIteratorRequest = requestBuilder.build(); final GetShardIteratorResponse getShardIteratorResponse = FutureUtils.resolveOrCancelFuture(kinesisClient.getShardIterator(getShardIteratorRequest), kinesisRequestTimeout); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java index fd1e045c..ee2850ed 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java @@ -21,6 +21,7 @@ import java.util.List; import lombok.EqualsAndHashCode; import lombok.NonNull; import lombok.ToString; +import software.amazon.awssdk.regions.Region; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; @@ -48,6 +49,10 @@ public class SingleStreamTracker implements StreamTracker { this(StreamIdentifier.singleStreamInstance(streamName)); } + public SingleStreamTracker(String streamName, Region region) { + this(StreamIdentifier.singleStreamInstance(streamName, region)); + } + public SingleStreamTracker(StreamIdentifier streamIdentifier) { this(streamIdentifier, DEFAULT_POSITION_IN_STREAM); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 8ada4970..d8fcf39e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.4.9-SNAPSHOT"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.0-SNAPSHOT"; /** * Client used to make calls to Kinesis for records retrieval @@ -120,7 +120,9 @@ public class RetrievalConfig { public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName, @NonNull String applicationName) { - this(kinesisAsyncClient, new SingleStreamTracker(streamName), applicationName); + this(kinesisAsyncClient, + new SingleStreamTracker(streamName, kinesisAsyncClient.serviceClientConfiguration().region()), + applicationName); } public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull StreamTracker streamTracker, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java index d17828e9..96f8f851 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java @@ -238,12 +238,15 @@ public class KinesisDataFetcher implements DataFetcher { GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder() .streamName(streamIdentifier.streamName()).shardId(shardId); + streamIdentifier.streamARNOptional().ifPresent(arn -> builder.streamARN(arn.toString())); + GetShardIteratorRequest request; if (isIteratorRestart) { request = IteratorBuilder.reconnectRequest(builder, sequenceNumber, initialPositionInStream).build(); } else { request = IteratorBuilder.request(builder, sequenceNumber, initialPositionInStream).build(); } + log.debug("[GetShardIterator] Request has parameters {}", request); // TODO: Check if this metric is fine to be added final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION); @@ -315,9 +318,11 @@ public class KinesisDataFetcher implements DataFetcher { } @Override - public GetRecordsRequest getGetRecordsRequest(String nextIterator) { - return KinesisRequestsBuilder.getRecordsRequestBuilder().shardIterator(nextIterator) - .limit(maxRecords).build(); + public GetRecordsRequest getGetRecordsRequest(String nextIterator) { + GetRecordsRequest.Builder builder = KinesisRequestsBuilder.getRecordsRequestBuilder() + .shardIterator(nextIterator).limit(maxRecords); + streamIdentifier.streamARNOptional().ifPresent(arn -> builder.streamARN(arn.toString())); + return builder.build(); } @Override diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java index 87caaa34..e1de0981 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java @@ -33,12 +33,12 @@ import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; +import software.amazon.kinesis.utils.MockObjectHelper; @RunWith(MockitoJUnitRunner.class) public class ConfigsBuilderTest { - @Mock - private KinesisAsyncClient mockKinesisClient; + private final KinesisAsyncClient mockKinesisClient = MockObjectHelper.createKinesisClient(); @Mock private DynamoDbAsyncClient mockDynamoClient; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java new file mode 100644 index 00000000..49c688ad --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java @@ -0,0 +1,113 @@ +package software.amazon.kinesis.common; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; + +import java.util.Optional; +import java.util.function.Supplier; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({ StreamARNUtil.class }) +public class StreamARNUtilTest { + private static final String STS_RESPONSE_ARN_FORMAT = "arn:aws:sts::%s:assumed-role/Admin/alias"; + private static final String KINESIS_STREAM_ARN_FORMAT = "arn:aws:kinesis:us-east-1:%s:stream/%s"; + // To prevent clashes in the stream arn cache with identical names, + // we're using the test name as the stream name (key) + private static final Supplier streamNameProvider = () -> Thread.currentThread().getStackTrace()[2].getMethodName(); + + @Mock + private StsClient mockStsClient; + + @Before + public void setUp() throws Exception { + MockitoAnnotations.initMocks(this); + PowerMockito.spy(StreamARNUtil.class); + PowerMockito.doReturn(mockStsClient).when(StreamARNUtil.class, "getStsClient"); + } + + @Test + public void testGetStreamARNHappyCase() { + String streamName = streamNameProvider.get(); + String accountId = "123456789012"; + when(mockStsClient.getCallerIdentity()) + .thenReturn(GetCallerIdentityResponse.builder().arn(String.format(STS_RESPONSE_ARN_FORMAT, accountId)).build()); + + Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1); + String expectedStreamARN = String.format(KINESIS_STREAM_ARN_FORMAT, accountId, streamName); + verify(mockStsClient, times(1)).getCallerIdentity(); + assertTrue(actualStreamARNOptional.isPresent()); + assertEquals(expectedStreamARN, actualStreamARNOptional.get().toString()); + } + + @Test + public void testGetStreamARNFromCache() { + String streamName = streamNameProvider.get(); + String accountId = "123456789012"; + when(mockStsClient.getCallerIdentity()) + .thenReturn(GetCallerIdentityResponse.builder().arn(String.format(STS_RESPONSE_ARN_FORMAT, accountId)).build()); + + Optional actualStreamARNOptional1 = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1); + Optional actualStreamARNOptional2 = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1); + String expectedStreamARN = String.format(KINESIS_STREAM_ARN_FORMAT, accountId, streamName); + // Since the second ARN is obtained from the cache, hence there's only one sts call + verify(mockStsClient, times(1)).getCallerIdentity(); + assertEquals(expectedStreamARN, actualStreamARNOptional1.get().toString()); + assertEquals(actualStreamARNOptional1, actualStreamARNOptional2); + } + + @Test + public void testGetStreamARNReturnsEmptyOnSTSError() { + // Optional.empty() is expected when there is an error with the STS call and STS returns empty Arn + String streamName = streamNameProvider.get(); + when(mockStsClient.getCallerIdentity()) + .thenThrow(AwsServiceException.builder().message("testAwsServiceException").build()) + .thenThrow(SdkClientException.builder().message("testSdkClientException").build()); + assertFalse(StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1).isPresent()); + assertFalse(StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1).isPresent()); + } + + @Test + public void testGetStreamARNReturnsEmptyOnInvalidKinesisRegion() { + // Optional.empty() is expected when kinesis region is not set correctly + String streamName = streamNameProvider.get(); + Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(streamName, null); + verify(mockStsClient, times(0)).getCallerIdentity(); + assertFalse(actualStreamARNOptional.isPresent()); + } + + @Test + public void testGetStreamARNWithProvidedAccountIDAndIgnoredSTSResult() throws Exception { + // If the account id is provided in the StreamIdentifier, it will override the result (account id) returned by sts + String streamName = streamNameProvider.get(); + String stsAccountId = "111111111111"; + String providedAccountId = "222222222222"; + when(mockStsClient.getCallerIdentity()) + .thenReturn(GetCallerIdentityResponse.builder().arn(String.format(STS_RESPONSE_ARN_FORMAT, stsAccountId)).build()); + + Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1, Optional.of(providedAccountId)); + String expectedStreamARN = String.format(KINESIS_STREAM_ARN_FORMAT, providedAccountId, streamName); + verify(mockStsClient, times(1)).getCallerIdentity(); + assertTrue(actualStreamARNOptional.isPresent()); + assertEquals(expectedStreamARN, actualStreamARNOptional.get().toString()); + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java new file mode 100644 index 00000000..d7a0b2ca --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java @@ -0,0 +1,94 @@ +package software.amazon.kinesis.common; + +import com.google.common.base.Joiner; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.regions.Region; + +import java.util.Optional; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mockStatic; + + +@RunWith(PowerMockRunner.class) +@PrepareForTest(StreamARNUtil.class) +public class StreamIdentifierTest { + private static final String streamName = "streamName"; + private static final Region kinesisRegion = Region.US_WEST_1; + private static final String accountId = "111111111111"; + private static final String epoch = "1680616058"; + + @Test + public void testSingleStreamInstanceWithName() { + StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(streamName); + Assert.assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); + Assert.assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); + Assert.assertFalse(actualStreamIdentifier.streamARNOptional().isPresent()); + Assert.assertEquals(streamName, actualStreamIdentifier.streamName()); + } + + @Test + public void testSingleStreamInstanceWithNameAndRegion() { + Optional arn = Optional.of(Arn.builder().partition("aws").service("kinesis") + .region(kinesisRegion.toString()).accountId("123").resource("stream/" + streamName).build()); + mockStatic(StreamARNUtil.class); + when(StreamARNUtil.getStreamARN(eq(streamName), eq(kinesisRegion))).thenReturn(arn); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(streamName, kinesisRegion); + Assert.assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); + Assert.assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); + Assert.assertTrue(actualStreamIdentifier.streamARNOptional().isPresent()); + Assert.assertEquals(arn, actualStreamIdentifier.streamARNOptional()); + } + + @Test + public void testMultiStreamInstanceWithIdentifierSerialization() { + String epoch = "1680616058"; + Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") + .accountId(accountId).region(kinesisRegion.toString()).resource("stream/" + streamName).build()); + + mockStatic(StreamARNUtil.class); + when(StreamARNUtil.getStreamARN(eq(streamName), any(), any())).thenReturn(arn); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(accountId, streamName, epoch, kinesisRegion)); + assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); + } + + @Test + public void testMultiStreamInstanceWithRegionSerialized() { + Region serializedRegion = Region.US_GOV_EAST_1; + Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") + .accountId(accountId).region(serializedRegion.toString()).resource("stream/" + streamName).build()); + + mockStatic(StreamARNUtil.class); + when(StreamARNUtil.getStreamARN(eq(streamName), eq(serializedRegion), any())).thenReturn(arn); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(accountId, streamName, epoch, serializedRegion), kinesisRegion); + assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); + } + + @Test + public void testMultiStreamInstanceWithoutRegionSerialized() { + Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") + .accountId(accountId).region(kinesisRegion.toString()).resource("stream/" + streamName).build()); + + mockStatic(StreamARNUtil.class); + when(StreamARNUtil.getStreamARN(eq(streamName), eq(kinesisRegion), any())).thenReturn(arn); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(accountId, streamName, epoch), kinesisRegion); + assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); + } + + private void assertActualStreamIdentifierExpected(Optional expectedARN, StreamIdentifier actual) { + Assert.assertTrue(actual.streamCreationEpochOptional().isPresent()); + Assert.assertTrue(actual.accountIdOptional().isPresent()); + Assert.assertTrue(actual.streamARNOptional().isPresent()); + Assert.assertEquals(expectedARN, actual.streamARNOptional()); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 2a0a1dae..aeb07e66 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -112,6 +112,7 @@ import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; import software.amazon.kinesis.retrieval.RetrievalFactory; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import software.amazon.kinesis.utils.MockObjectHelper; /** * @@ -137,7 +138,6 @@ public class SchedulerTest { private ProcessorConfig processorConfig; private RetrievalConfig retrievalConfig; - @Mock private KinesisAsyncClient kinesisClient; @Mock private DynamoDbAsyncClient dynamoDBClient; @@ -180,6 +180,7 @@ public class SchedulerTest { lifecycleConfig = new LifecycleConfig(); metricsConfig = new MetricsConfig(cloudWatchClient, namespace); processorConfig = new ProcessorConfig(shardRecordProcessorFactory); + kinesisClient = MockObjectHelper.createKinesisClient(); retrievalConfig = new RetrievalConfig(kinesisClient, streamName, applicationName) .retrievalFactory(retrievalFactory); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java index 464459d5..49cf4673 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java @@ -24,13 +24,13 @@ import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; +import software.amazon.kinesis.utils.MockObjectHelper; @RunWith(MockitoJUnitRunner.class) public class RetrievalConfigTest { private static final String APPLICATION_NAME = RetrievalConfigTest.class.getSimpleName(); - @Mock private KinesisAsyncClient mockKinesisClient; @Mock @@ -38,6 +38,7 @@ public class RetrievalConfigTest { @Before public void setUp() { + mockKinesisClient = MockObjectHelper.createKinesisClient(true); when(mockMultiStreamTracker.isMultiStream()).thenReturn(true); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java new file mode 100644 index 00000000..4fd050be --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java @@ -0,0 +1,31 @@ +package software.amazon.kinesis.utils; + +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisServiceClientConfiguration; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public final class MockObjectHelper { + + public static KinesisAsyncClient createKinesisClient() { + return createKinesisClient(Region.US_EAST_1); + } + + /** + * @param isRegionDummy a boolean to determine whether to use a null value for the Kinesis client's region. + * @return + */ + public static KinesisAsyncClient createKinesisClient(boolean isRegionDummy) { + return isRegionDummy ? createKinesisClient(null) : createKinesisClient(); + } + + public static KinesisAsyncClient createKinesisClient(Region region) { + KinesisAsyncClient kinesisClient = mock(KinesisAsyncClient.class); + when(kinesisClient.serviceClientConfiguration()). + thenReturn(KinesisServiceClientConfiguration.builder().region(region).build()); + return kinesisClient; + } + +} diff --git a/pom.xml b/pom.xml index 5c41c20e..fc3ad7a2 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.4.9-SNAPSHOT + 2.5.0-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 0fd94acb2bbe972d3608e6bf0a16d119a43b399b Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Tue, 18 Apr 2023 16:29:24 -0400 Subject: [PATCH 54/88] StreamARN: fast-follow to invent-and-simplify https://github.com/awslabs/amazon-kinesis-client/pull/1087 (#1097) * fixed memory leak in `StreamARNUtil` (new class) * substantial DRY * added more, and enhanced recently-provided, unit tests --- .../amazon/kinesis/common/StreamARNUtil.java | 74 ++++----- .../kinesis/common/StreamARNUtilTest.java | 157 +++++++++++++----- .../kinesis/common/StreamIdentifierTest.java | 91 +++++----- 3 files changed, 187 insertions(+), 135 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java index 7fe2d435..03652a5b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java @@ -1,86 +1,68 @@ package software.amazon.kinesis.common; -import com.google.common.base.Joiner; import lombok.AccessLevel; import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.awscore.exception.AwsServiceException; import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.http.SdkHttpClient; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; -import java.util.HashMap; import java.util.Optional; @Slf4j @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class StreamARNUtil { - private static final HashMap streamARNCache = new HashMap<>(); + + /** + * Caches an {@link Arn} constructed from a {@link StsClient#getCallerIdentity()} call. + */ + private static final SupplierCache CALLER_IDENTITY_ARN = new SupplierCache<>(() -> { + try (final SdkHttpClient httpClient = UrlConnectionHttpClient.builder().build(); + final StsClient stsClient = StsClient.builder().httpClient(httpClient).build()) { + final GetCallerIdentityResponse response = stsClient.getCallerIdentity(); + return Arn.fromString(response.arn()); + } catch (AwsServiceException | SdkClientException e) { + log.warn("Unable to get sts caller identity to build stream arn", e); + return null; + } + }); /** * This static method attempts to retrieve the stream ARN using the stream name, region, and accountId returned by STS * It is designed to fail gracefully, returning Optional.empty() if any errors occur. - * @param streamName: stream name - * @param kinesisRegion: kinesisRegion is a nullable parameter used to construct the stream arn - * @return + * + * @param streamName stream name + * @param kinesisRegion kinesisRegion is a nullable parameter used to construct the stream arn */ public static Optional getStreamARN(String streamName, Region kinesisRegion) { return getStreamARN(streamName, kinesisRegion, Optional.empty()); } public static Optional getStreamARN(String streamName, Region kinesisRegion, @NonNull Optional accountId) { - if (kinesisRegion == null || StringUtils.isEmpty(kinesisRegion.toString())) { + if (kinesisRegion == null) { return Optional.empty(); } - // Consult the cache before contacting STS - String key = getCacheKey(streamName, kinesisRegion, accountId); - if (streamARNCache.containsKey(key)) { - return Optional.of(streamARNCache.get(key)); - } - Optional stsCallerArn = getStsCallerArn(); - if (!stsCallerArn.isPresent() || !stsCallerArn.get().accountId().isPresent()) { + final Arn identityArn = CALLER_IDENTITY_ARN.get(); + if (identityArn == null) { return Optional.empty(); } - accountId = accountId.isPresent() ? accountId : stsCallerArn.get().accountId(); - Arn kinesisStreamArn = Arn.builder() - .partition(stsCallerArn.get().partition()) + + // the provided accountId takes precedence + final String chosenAccountId = accountId.orElse(identityArn.accountId().orElse("")); + return Optional.of(Arn.builder() + .partition(identityArn.partition()) .service("kinesis") .region(kinesisRegion.toString()) - .accountId(accountId.get()) + .accountId(chosenAccountId) .resource("stream/" + streamName) - .build(); - - // Update the cache - streamARNCache.put(key, kinesisStreamArn); - return Optional.of(kinesisStreamArn); - } - - private static Optional getStsCallerArn() { - GetCallerIdentityResponse response; - try { - response = getStsClient().getCallerIdentity(); - } catch (AwsServiceException | SdkClientException e) { - log.warn("Unable to get sts caller identity to build stream arn", e); - return Optional.empty(); - } - return Optional.of(Arn.fromString(response.arn())); - } - - private static StsClient getStsClient() { - return StsClient.builder() - .httpClient(UrlConnectionHttpClient.builder().build()) - .build(); - } - - private static String getCacheKey( - String streamName, @NonNull Region kinesisRegion, @NonNull Optional accountId) { - return Joiner.on(":").join(streamName, kinesisRegion.toString(), accountId.orElse("")); + .build()); } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java index 49c688ad..595710c6 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java @@ -8,106 +8,173 @@ import org.mockito.MockitoAnnotations; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.awscore.exception.AwsServiceException; import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sts.StsClient; -import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; +import software.amazon.awssdk.services.sts.StsClientBuilder; +import java.lang.reflect.Field; import java.util.Optional; -import java.util.function.Supplier; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; @RunWith(PowerMockRunner.class) -@PrepareForTest({ StreamARNUtil.class }) +@PrepareForTest({ StreamARNUtil.class, StsClient.class, UrlConnectionHttpClient.class }) public class StreamARNUtilTest { private static final String STS_RESPONSE_ARN_FORMAT = "arn:aws:sts::%s:assumed-role/Admin/alias"; private static final String KINESIS_STREAM_ARN_FORMAT = "arn:aws:kinesis:us-east-1:%s:stream/%s"; - // To prevent clashes in the stream arn cache with identical names, - // we're using the test name as the stream name (key) - private static final Supplier streamNameProvider = () -> Thread.currentThread().getStackTrace()[2].getMethodName(); + + /** + * Original {@link SupplierCache} that is constructed on class load. + */ + private static final SupplierCache ORIGINAL_CACHE = Whitebox.getInternalState( + StreamARNUtil.class, "CALLER_IDENTITY_ARN"); + + private static final String ACCOUNT_ID = "12345"; + + private static final String STREAM_NAME = StreamARNUtilTest.class.getSimpleName(); + + @Mock + private StsClientBuilder mockStsClientBuilder; @Mock private StsClient mockStsClient; + private SupplierCache spySupplierCache; + @Before public void setUp() throws Exception { MockitoAnnotations.initMocks(this); - PowerMockito.spy(StreamARNUtil.class); - PowerMockito.doReturn(mockStsClient).when(StreamARNUtil.class, "getStsClient"); + + setUpSupplierCache(); + + final Arn defaultArn = toArn(STS_RESPONSE_ARN_FORMAT, ACCOUNT_ID); + doReturn(defaultArn).when(spySupplierCache).get(); + } + + private void setUpSts() { + PowerMockito.mockStatic(StsClient.class); + PowerMockito.mockStatic(UrlConnectionHttpClient.class); + + when(UrlConnectionHttpClient.builder()).thenReturn(mock(UrlConnectionHttpClient.Builder.class)); + when(StsClient.builder()).thenReturn(mockStsClientBuilder); + when(mockStsClientBuilder.httpClient(any(SdkHttpClient.class))).thenReturn(mockStsClientBuilder); + when(mockStsClientBuilder.build()).thenReturn(mockStsClient); + + // bypass the spy so the Sts clients are called + when(spySupplierCache.get()).thenCallRealMethod(); + } + + /** + * Wrap and embed the original {@link SupplierCache} with a spy to avoid + * one-and-done cache behavior, provide each test precise control over + * return values, and enable the ability to verify interactions via Mockito. + */ + private void setUpSupplierCache() throws Exception { + spySupplierCache = spy(ORIGINAL_CACHE); + + final Field f = StreamARNUtil.class.getDeclaredField("CALLER_IDENTITY_ARN"); + f.setAccessible(true); + f.set(null, spySupplierCache); + f.setAccessible(false); } @Test public void testGetStreamARNHappyCase() { - String streamName = streamNameProvider.get(); - String accountId = "123456789012"; - when(mockStsClient.getCallerIdentity()) - .thenReturn(GetCallerIdentityResponse.builder().arn(String.format(STS_RESPONSE_ARN_FORMAT, accountId)).build()); + getStreamArn(); - Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1); - String expectedStreamARN = String.format(KINESIS_STREAM_ARN_FORMAT, accountId, streamName); - verify(mockStsClient, times(1)).getCallerIdentity(); - assertTrue(actualStreamARNOptional.isPresent()); - assertEquals(expectedStreamARN, actualStreamARNOptional.get().toString()); + verify(spySupplierCache).get(); } @Test public void testGetStreamARNFromCache() { - String streamName = streamNameProvider.get(); - String accountId = "123456789012"; - when(mockStsClient.getCallerIdentity()) - .thenReturn(GetCallerIdentityResponse.builder().arn(String.format(STS_RESPONSE_ARN_FORMAT, accountId)).build()); + final Optional actualStreamARNOptional1 = getStreamArn(); + final Optional actualStreamARNOptional2 = getStreamArn(); - Optional actualStreamARNOptional1 = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1); - Optional actualStreamARNOptional2 = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1); - String expectedStreamARN = String.format(KINESIS_STREAM_ARN_FORMAT, accountId, streamName); - // Since the second ARN is obtained from the cache, hence there's only one sts call - verify(mockStsClient, times(1)).getCallerIdentity(); - assertEquals(expectedStreamARN, actualStreamARNOptional1.get().toString()); + verify(spySupplierCache, times(2)).get(); assertEquals(actualStreamARNOptional1, actualStreamARNOptional2); } @Test public void testGetStreamARNReturnsEmptyOnSTSError() { + setUpSts(); + // Optional.empty() is expected when there is an error with the STS call and STS returns empty Arn - String streamName = streamNameProvider.get(); when(mockStsClient.getCallerIdentity()) .thenThrow(AwsServiceException.builder().message("testAwsServiceException").build()) .thenThrow(SdkClientException.builder().message("testSdkClientException").build()); - assertFalse(StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1).isPresent()); - assertFalse(StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1).isPresent()); + + assertEquals(Optional.empty(), StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1)); + assertEquals(Optional.empty(), StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1)); + verify(mockStsClient, times(2)).getCallerIdentity(); + verify(spySupplierCache, times(2)).get(); } @Test public void testGetStreamARNReturnsEmptyOnInvalidKinesisRegion() { // Optional.empty() is expected when kinesis region is not set correctly - String streamName = streamNameProvider.get(); - Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(streamName, null); - verify(mockStsClient, times(0)).getCallerIdentity(); - assertFalse(actualStreamARNOptional.isPresent()); + Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(STREAM_NAME, null); + assertEquals(Optional.empty(), actualStreamARNOptional); + verifyZeroInteractions(mockStsClient); + verifyZeroInteractions(spySupplierCache); } @Test - public void testGetStreamARNWithProvidedAccountIDAndIgnoredSTSResult() throws Exception { + public void testGetStreamARNWithProvidedAccountIDAndIgnoredSTSResult() { // If the account id is provided in the StreamIdentifier, it will override the result (account id) returned by sts - String streamName = streamNameProvider.get(); - String stsAccountId = "111111111111"; - String providedAccountId = "222222222222"; - when(mockStsClient.getCallerIdentity()) - .thenReturn(GetCallerIdentityResponse.builder().arn(String.format(STS_RESPONSE_ARN_FORMAT, stsAccountId)).build()); + final String cachedAccountId = "111111111111"; + final String providedAccountId = "222222222222"; - Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(streamName, Region.US_EAST_1, Optional.of(providedAccountId)); - String expectedStreamARN = String.format(KINESIS_STREAM_ARN_FORMAT, providedAccountId, streamName); - verify(mockStsClient, times(1)).getCallerIdentity(); + final Arn cachedArn = toArn(STS_RESPONSE_ARN_FORMAT, cachedAccountId); + when(spySupplierCache.get()).thenReturn(cachedArn); + + final Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1, + Optional.of(providedAccountId)); + final Arn expectedStreamARN = toArn(KINESIS_STREAM_ARN_FORMAT, providedAccountId, STREAM_NAME); + + verify(spySupplierCache).get(); + verifyZeroInteractions(mockStsClient); assertTrue(actualStreamARNOptional.isPresent()); - assertEquals(expectedStreamARN, actualStreamARNOptional.get().toString()); + assertEquals(expectedStreamARN, actualStreamARNOptional.get()); + } + + @Test + public void testNoAccountId() { + final Arn arnWithoutAccountId = toArn(STS_RESPONSE_ARN_FORMAT, ""); + when(spySupplierCache.get()).thenReturn(arnWithoutAccountId); + assertEquals(Optional.empty(), arnWithoutAccountId.accountId()); + + final Optional actualArn = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); + assertTrue(actualArn.isPresent()); + assertEquals(Optional.empty(), actualArn.get().accountId()); + } + + private static Optional getStreamArn() { + final Optional actualArn = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); + final Arn expectedArn = toArn(KINESIS_STREAM_ARN_FORMAT, ACCOUNT_ID, STREAM_NAME); + + assertTrue(actualArn.isPresent()); + assertEquals(expectedArn, actualArn.get()); + + return actualArn; + } + + private static Arn toArn(final String format, final Object... params) { + return Arn.fromString(String.format(format, params)); } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java index d7a0b2ca..3c4b5e34 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java @@ -1,7 +1,6 @@ package software.amazon.kinesis.common; -import com.google.common.base.Joiner; -import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -11,84 +10,88 @@ import software.amazon.awssdk.regions.Region; import java.util.Optional; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.mockStatic; - @RunWith(PowerMockRunner.class) @PrepareForTest(StreamARNUtil.class) public class StreamIdentifierTest { - private static final String streamName = "streamName"; - private static final Region kinesisRegion = Region.US_WEST_1; - private static final String accountId = "111111111111"; - private static final String epoch = "1680616058"; + private static final String STREAM_NAME = "streamName"; + private static final Region KINESIS_REGION = Region.US_WEST_1; + private static final String TEST_ACCOUNT_ID = "111111111111"; + private static final String EPOCH = "1680616058"; + + private static final Arn DEFAULT_ARN = Arn.builder().partition("aws").service("kinesis") + .region(KINESIS_REGION.toString()).accountId(TEST_ACCOUNT_ID).resource("stream/" + STREAM_NAME).build(); + + @Before + public void setUp() { + mockStatic(StreamARNUtil.class); + + when(StreamARNUtil.getStreamARN(anyString(), any(Region.class))).thenReturn(Optional.empty()); + when(StreamARNUtil.getStreamARN(STREAM_NAME, KINESIS_REGION)).thenReturn(Optional.of(DEFAULT_ARN)); + when(StreamARNUtil.getStreamARN(STREAM_NAME, KINESIS_REGION, Optional.of(TEST_ACCOUNT_ID))) + .thenReturn(Optional.of(DEFAULT_ARN)); + } @Test public void testSingleStreamInstanceWithName() { - StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(streamName); - Assert.assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); - Assert.assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); - Assert.assertFalse(actualStreamIdentifier.streamARNOptional().isPresent()); - Assert.assertEquals(streamName, actualStreamIdentifier.streamName()); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME); + assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); + assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); + assertFalse(actualStreamIdentifier.streamARNOptional().isPresent()); + assertEquals(STREAM_NAME, actualStreamIdentifier.streamName()); } @Test public void testSingleStreamInstanceWithNameAndRegion() { - Optional arn = Optional.of(Arn.builder().partition("aws").service("kinesis") - .region(kinesisRegion.toString()).accountId("123").resource("stream/" + streamName).build()); - mockStatic(StreamARNUtil.class); - when(StreamARNUtil.getStreamARN(eq(streamName), eq(kinesisRegion))).thenReturn(arn); - StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(streamName, kinesisRegion); - Assert.assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); - Assert.assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); - Assert.assertTrue(actualStreamIdentifier.streamARNOptional().isPresent()); - Assert.assertEquals(arn, actualStreamIdentifier.streamARNOptional()); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME, KINESIS_REGION); + assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); + assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); + assertTrue(actualStreamIdentifier.streamARNOptional().isPresent()); + assertEquals(DEFAULT_ARN, actualStreamIdentifier.streamARNOptional().get()); } @Test public void testMultiStreamInstanceWithIdentifierSerialization() { - String epoch = "1680616058"; - Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") - .accountId(accountId).region(kinesisRegion.toString()).resource("stream/" + streamName).build()); - - mockStatic(StreamARNUtil.class); - when(StreamARNUtil.getStreamARN(eq(streamName), any(), any())).thenReturn(arn); StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(accountId, streamName, epoch, kinesisRegion)); - assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); + String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, EPOCH, KINESIS_REGION.toString())); + assertActualStreamIdentifierExpected(DEFAULT_ARN, actualStreamIdentifier); } @Test public void testMultiStreamInstanceWithRegionSerialized() { Region serializedRegion = Region.US_GOV_EAST_1; Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") - .accountId(accountId).region(serializedRegion.toString()).resource("stream/" + streamName).build()); + .accountId(TEST_ACCOUNT_ID).region(serializedRegion.toString()).resource("stream/" + STREAM_NAME).build()); - mockStatic(StreamARNUtil.class); - when(StreamARNUtil.getStreamARN(eq(streamName), eq(serializedRegion), any())).thenReturn(arn); + when(StreamARNUtil.getStreamARN(eq(STREAM_NAME), eq(serializedRegion), any(Optional.class))).thenReturn(arn); StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(accountId, streamName, epoch, serializedRegion), kinesisRegion); + String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, EPOCH, serializedRegion.toString()), KINESIS_REGION); assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); } @Test public void testMultiStreamInstanceWithoutRegionSerialized() { - Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") - .accountId(accountId).region(kinesisRegion.toString()).resource("stream/" + streamName).build()); - - mockStatic(StreamARNUtil.class); - when(StreamARNUtil.getStreamARN(eq(streamName), eq(kinesisRegion), any())).thenReturn(arn); StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(accountId, streamName, epoch), kinesisRegion); - assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); + String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, EPOCH), KINESIS_REGION); + assertActualStreamIdentifierExpected(DEFAULT_ARN, actualStreamIdentifier); } private void assertActualStreamIdentifierExpected(Optional expectedARN, StreamIdentifier actual) { - Assert.assertTrue(actual.streamCreationEpochOptional().isPresent()); - Assert.assertTrue(actual.accountIdOptional().isPresent()); - Assert.assertTrue(actual.streamARNOptional().isPresent()); - Assert.assertEquals(expectedARN, actual.streamARNOptional()); + assertActualStreamIdentifierExpected(expectedARN.get(), actual); + } + + private void assertActualStreamIdentifierExpected(Arn expectedARN, StreamIdentifier actual) { + assertTrue(actual.streamCreationEpochOptional().isPresent()); + assertTrue(actual.accountIdOptional().isPresent()); + assertTrue(actual.streamARNOptional().isPresent()); + assertEquals(Optional.of(expectedARN), actual.streamARNOptional()); } } From b86fa22e9641dc561d979132e1c1cf65f3a53240 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Wed, 19 Apr 2023 19:38:00 -0400 Subject: [PATCH 55/88] =?UTF-8?q?StreamARN:=20enhanced=20`StreamIdentifier?= =?UTF-8?q?`=20to=20spawn=20instances=20from=20stream=E2=80=A6=20(#1098)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * StreamARN: enhanced `StreamIdentifier` to spawn instances from stream ARNs. * Bugfix: moved instantation of `Future` inside `Supplier`. --- .../kinesis/common/StreamIdentifier.java | 187 +++++++++++++----- .../retrieval/KinesisClientFacade.java | 100 ++++++++++ .../kinesis/retrieval/RetrievalConfig.java | 3 +- .../kinesis/common/StreamIdentifierTest.java | 152 +++++++++++--- .../PeriodicShardSyncManagerTest.java | 2 +- .../kinesis/coordinator/SchedulerTest.java | 12 +- .../leases/HierarchicalShardSyncerTest.java | 2 +- .../retrieval/KinesisClientFacadeTest.java | 66 +++++++ .../retrieval/fanout/FanOutConfigTest.java | 4 +- 9 files changed, 443 insertions(+), 85 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java index 0fc7d2b2..1a81f606 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -15,46 +15,89 @@ package software.amazon.kinesis.common; -import com.google.common.base.Joiner; import lombok.AccessLevel; import lombok.Builder; import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.ToString; +import lombok.NonNull; import lombok.experimental.Accessors; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; import software.amazon.awssdk.utils.Validate; +import software.amazon.kinesis.retrieval.KinesisClientFacade; import java.util.Optional; +import java.util.regex.Matcher; import java.util.regex.Pattern; @Builder(access = AccessLevel.PRIVATE) @EqualsAndHashCode @Getter -@ToString @Accessors(fluent = true) public class StreamIdentifier { + @Builder.Default private final Optional accountIdOptional = Optional.empty(); + @NonNull private final String streamName; @Builder.Default - private final Optional streamCreationEpochOptional = Optional.empty(); + private Optional streamCreationEpochOptional = Optional.empty(); @Builder.Default private final Optional streamARNOptional = Optional.empty(); - private static final String DELIMITER = ":"; - private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*" + ":?([a-z]{2}(-gov)?-[a-z]+-\\d{1})?"); + /** + * Pattern for a serialized {@link StreamIdentifier}. The valid format is + * {@code ::[:]} where + * {@code region} is the id representation of a {@link Region} and is + * optional. + */ + private static final Pattern STREAM_IDENTIFIER_PATTERN = Pattern.compile( + // `?::` has two parts: `?:` starts a non-capturing group, and + // `:` is the first character in the group (i.e., ":") + "(?[0-9]+):(?[^:]+):(?[0-9]+)(?::(?[-a-z0-9]+))?"); + + /** + * Pattern for a stream ARN. The valid format is + * {@code arn:aws:kinesis:::stream:} + * where {@code region} is the id representation of a {@link Region}. + */ + private static final Pattern STREAM_ARN_PATTERN = Pattern.compile( + "arn:aws:kinesis:(?[-a-z0-9]+):(?[0-9]{12}):stream/(?.+)"); /** * Serialize the current StreamIdentifier instance. - * TODO: Consider appending region info for cross-account consumer support - * @return + * + * @return a String of {@code account:stream:creationEpoch[:region]} + * where {@code region} is the id representation of a {@link Region} + * and is optional. */ public String serialize() { - return accountIdOptional.isPresent() ? - Joiner.on(DELIMITER).join(accountIdOptional.get(), streamName, streamCreationEpochOptional.get()) : - streamName; + if (!accountIdOptional.isPresent()) { + return streamName; + } + + if (!streamCreationEpochOptional.isPresent()) { + // FIXME bias-for-action hack to simplify back-porting into KCL 1.x and facilitate the + // backwards-compatible requirement. There's a chicken-and-egg issue if DSS is + // called as the application is being configured (and before the client is rigged). + // Furthermore, if epoch isn't lazy-loaded here, the problem quickly spirals into + // systemic issues of concurrency and consistency (e.g., PeriodicShardSyncManager, + // Scheduler, DDB leases). We should look at leveraging dependency injection. + // (NOTE: not to inject the Kinesis client here, but to ensure the client is + // accessible elsewhere ASAP.) + final DescribeStreamSummaryResponse dss = KinesisClientFacade.describeStreamSummary( + streamARNOptional().get().toString()); + final long creationEpoch = dss.streamDescriptionSummary().streamCreationTimestamp().getEpochSecond(); + streamCreationEpochOptional = Optional.of(creationEpoch); + } + + final char delimiter = ':'; + final StringBuilder sb = new StringBuilder(accountIdOptional.get()).append(delimiter) + .append(streamName).append(delimiter); + streamCreationEpochOptional.ifPresent(sb::append); + streamARNOptional.flatMap(Arn::region).ifPresent(region -> sb.append(delimiter).append(region)); + return sb.toString(); } @Override @@ -64,61 +107,113 @@ public class StreamIdentifier { /** * Create a multi stream instance for StreamIdentifier from serialized stream identifier. - * See the format of a serialized stream identifier at {@link StreamIdentifier#multiStreamInstance(String, Region)} - * @param streamIdentifierSer - * @return StreamIdentifier + * + * @param serializationOrArn serialized {@link StreamIdentifier} or AWS ARN of a Kinesis stream + * + * @see #multiStreamInstance(String, Region) + * @see #serialize() */ - public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) { - return multiStreamInstance(streamIdentifierSer, null); + public static StreamIdentifier multiStreamInstance(String serializationOrArn) { + return multiStreamInstance(serializationOrArn, null); } /** * Create a multi stream instance for StreamIdentifier from serialized stream identifier. - * @param streamIdentifierSer The serialized stream identifier should be of the format - * account:stream:creationepoch[:region] + * + * @param serializationOrArn serialized {@link StreamIdentifier} or AWS ARN of a Kinesis stream * @param kinesisRegion This nullable region is used to construct the optional StreamARN - * @return StreamIdentifier + * + * @see #serialize() */ - public static StreamIdentifier multiStreamInstance(String streamIdentifierSer, Region kinesisRegion) { - if (PATTERN.matcher(streamIdentifierSer).matches()) { - final String[] split = streamIdentifierSer.split(DELIMITER); - final String streamName = split[1]; - final Optional accountId = Optional.ofNullable(split[0]); - StreamIdentifierBuilder builder = StreamIdentifier.builder() - .accountIdOptional(accountId) - .streamName(streamName) - .streamCreationEpochOptional(Optional.of(Long.parseLong(split[2]))); - final Region region = (split.length == 4) ? - Region.of(split[3]) : // Use the region extracted from the serialized string, which matches the regex pattern - kinesisRegion; // Otherwise just use the provided region - final Optional streamARN = StreamARNUtil.getStreamARN(streamName, region, accountId); - return builder.streamARNOptional(streamARN).build(); - } else { - throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer); + public static StreamIdentifier multiStreamInstance(String serializationOrArn, Region kinesisRegion) { + final StreamIdentifier fromSerialization = fromSerialization(serializationOrArn, kinesisRegion); + if (fromSerialization != null) { + return fromSerialization; } + final StreamIdentifier fromArn = fromArn(serializationOrArn, kinesisRegion); + if (fromArn != null) { + return fromArn; + } + + throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + serializationOrArn); } /** * Create a single stream instance for StreamIdentifier from stream name. - * @param streamName - * @return StreamIdentifier + * + * @param streamNameOrArn stream name or AWS ARN of a Kinesis stream */ - public static StreamIdentifier singleStreamInstance(String streamName) { - return singleStreamInstance(streamName, null); + public static StreamIdentifier singleStreamInstance(String streamNameOrArn) { + return singleStreamInstance(streamNameOrArn, null); } /** * Create a single stream instance for StreamIdentifier from the provided stream name and kinesisRegion. * This method also constructs the optional StreamARN based on the region info. - * @param streamName - * @param kinesisRegion - * @return StreamIdentifier + * + * @param streamNameOrArn stream name or AWS ARN of a Kinesis stream + * @param kinesisRegion (optional) region used to construct the ARN */ - public static StreamIdentifier singleStreamInstance(String streamName, Region kinesisRegion) { - Validate.notEmpty(streamName, "StreamName should not be empty"); + public static StreamIdentifier singleStreamInstance(String streamNameOrArn, Region kinesisRegion) { + Validate.notEmpty(streamNameOrArn, "StreamName should not be empty"); + + final StreamIdentifier fromArn = fromArn(streamNameOrArn, kinesisRegion); + if (fromArn != null) { + return fromArn; + } + return StreamIdentifier.builder() - .streamName(streamName) - .streamARNOptional(StreamARNUtil.getStreamARN(streamName, kinesisRegion)) + .streamName(streamNameOrArn) + .streamARNOptional(StreamARNUtil.getStreamARN(streamNameOrArn, kinesisRegion)) .build(); } + + /** + * Deserializes a StreamIdentifier from {@link #STREAM_IDENTIFIER_PATTERN}. + * + * @param input input string (e.g., ARN, serialized instance) to convert into an instance + * @param kinesisRegion (optional) region used to construct the ARN + * @return a StreamIdentifier instance if the pattern matched, otherwise null + */ + private static StreamIdentifier fromSerialization(final String input, final Region kinesisRegion) { + final Matcher matcher = STREAM_IDENTIFIER_PATTERN.matcher(input); + return matcher.matches() + ? toStreamIdentifier(matcher, matcher.group("creationEpoch"), kinesisRegion) : null; + } + + /** + * Constructs a StreamIdentifier from {@link #STREAM_ARN_PATTERN}. + * + * @param input input string (e.g., ARN, serialized instance) to convert into an instance + * @param kinesisRegion (optional) region used to construct the ARN + * @return a StreamIdentifier instance if the pattern matched, otherwise null + */ + private static StreamIdentifier fromArn(final String input, final Region kinesisRegion) { + final Matcher matcher = STREAM_ARN_PATTERN.matcher(input); + return matcher.matches() + ? toStreamIdentifier(matcher, "", kinesisRegion) : null; + } + + private static StreamIdentifier toStreamIdentifier(final Matcher matcher, final String matchedEpoch, + final Region kinesisRegion) { + final Optional accountId = Optional.of(matcher.group("accountId")); + final String streamName = matcher.group("streamName"); + final Optional creationEpoch = matchedEpoch.isEmpty() ? Optional.empty() + : Optional.of(Long.valueOf(matchedEpoch)); + final String matchedRegion = matcher.group("region"); + final Region region = (matchedRegion != null) ? Region.of(matchedRegion) : kinesisRegion; + final Optional arn = StreamARNUtil.getStreamARN(streamName, region, accountId); + + if (!creationEpoch.isPresent() && !arn.isPresent()) { + throw new IllegalArgumentException("Cannot create StreamIdentifier if missing both ARN and creation epoch"); + } + + return StreamIdentifier.builder() + .accountIdOptional(accountId) + .streamName(streamName) + .streamCreationEpochOptional(creationEpoch) + .streamARNOptional(arn) + .build(); + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java new file mode 100644 index 00000000..5c9e732b --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java @@ -0,0 +1,100 @@ +package software.amazon.kinesis.retrieval; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; +import software.amazon.awssdk.services.kinesis.model.KinesisException; +import software.amazon.awssdk.services.kinesis.model.LimitExceededException; +import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; +import software.amazon.kinesis.common.KinesisRequestsBuilder; + +/** + * Facade pattern to simplify interactions with a {@link KinesisAsyncClient}. + */ +@Slf4j +public final class KinesisClientFacade { + + /** + * Reusable {@link AWSExceptionManager}. + *

+ * N.B. This instance is mutable, but thread-safe for read-only use. + *

+ */ + private static final AWSExceptionManager AWS_EXCEPTION_MANAGER; + + // FIXME dependency injection + private static KinesisAsyncClient kinesisClient; + + static { + AWS_EXCEPTION_MANAGER = new AWSExceptionManager(); + AWS_EXCEPTION_MANAGER.add(KinesisException.class, t -> t); + AWS_EXCEPTION_MANAGER.add(LimitExceededException.class, t -> t); + AWS_EXCEPTION_MANAGER.add(ResourceInUseException.class, t -> t); + AWS_EXCEPTION_MANAGER.add(ResourceNotFoundException.class, t -> t); + } + + static void initialize(final KinesisAsyncClient client) { + kinesisClient = client; + } + + public static DescribeStreamSummaryResponse describeStreamSummary(final String streamArn) { + final DescribeStreamSummaryRequest request = KinesisRequestsBuilder + .describeStreamSummaryRequestBuilder().streamARN(streamArn).build(); + final ServiceCallerSupplier dss = + () -> kinesisClient.describeStreamSummary(request).get(); + return retryWhenThrottled(dss, 3, streamArn, "DescribeStreamSummary"); + } + + // FIXME code lifted-and-shifted from FanOutConsumerRegistration; that class + // (and others) should not be responsible for interacting directly with + // the thread-safe Kinesis client (and handling retries, etc.) + private static T retryWhenThrottled( + @NonNull final ServiceCallerSupplier retriever, + final int maxRetries, + final String streamArn, + @NonNull final String apiName) { + LimitExceededException finalException = null; + + int retries = maxRetries; + while (retries > 0) { + try { + try { + return retriever.get(); + } catch (ExecutionException e) { + throw AWS_EXCEPTION_MANAGER.apply(e.getCause()); + } catch (InterruptedException e) { + throw KinesisException.create("Unable to complete " + apiName, e); + } catch (TimeoutException te) { + log.info("Timed out waiting for " + apiName + " for " + streamArn); + } + } catch (LimitExceededException e) { + log.info("{} : Throttled while calling {} API, will backoff.", streamArn, apiName); + try { + Thread.sleep(1000 + (long) (Math.random() * 100)); + } catch (InterruptedException ie) { + log.debug("Sleep interrupted, shutdown invoked."); + } + finalException = e; + } + retries--; + } + + if (finalException == null) { + throw new IllegalStateException(streamArn + " : Exhausted retries while calling " + apiName); + } + + throw finalException; + } + + @FunctionalInterface + private interface ServiceCallerSupplier { + T get() throws ExecutionException, InterruptedException, TimeoutException; + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index d8fcf39e..153faf70 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -33,7 +33,6 @@ import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; import software.amazon.kinesis.retrieval.fanout.FanOutConfig; -import software.amazon.kinesis.retrieval.polling.PollingConfig; /** * Used by the KCL to configure the retrieval of records from Kinesis. @@ -132,6 +131,8 @@ public class RetrievalConfig { this.applicationName = applicationName; this.appStreamTracker = DeprecationUtils.convert(streamTracker, singleStreamTracker -> singleStreamTracker.streamConfigList().get(0)); + + KinesisClientFacade.initialize(kinesisAsyncClient); } /** diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java index 3c4b5e34..115cab03 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java @@ -1,5 +1,6 @@ package software.amazon.kinesis.common; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -8,38 +9,113 @@ import org.powermock.modules.junit4.PowerMockRunner; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.regions.Region; +import java.util.Arrays; import java.util.Optional; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.verifyStatic; +import static software.amazon.kinesis.common.StreamARNUtil.getStreamARN; @RunWith(PowerMockRunner.class) @PrepareForTest(StreamARNUtil.class) public class StreamIdentifierTest { - private static final String STREAM_NAME = "streamName"; + private static final String STREAM_NAME = "stream-name"; private static final Region KINESIS_REGION = Region.US_WEST_1; - private static final String TEST_ACCOUNT_ID = "111111111111"; - private static final String EPOCH = "1680616058"; + private static final String TEST_ACCOUNT_ID = "123456789012"; + private static final long EPOCH = 1680616058L; - private static final Arn DEFAULT_ARN = Arn.builder().partition("aws").service("kinesis") - .region(KINESIS_REGION.toString()).accountId(TEST_ACCOUNT_ID).resource("stream/" + STREAM_NAME).build(); + private static final Arn DEFAULT_ARN = toArn(KINESIS_REGION); @Before public void setUp() { mockStatic(StreamARNUtil.class); - when(StreamARNUtil.getStreamARN(anyString(), any(Region.class))).thenReturn(Optional.empty()); - when(StreamARNUtil.getStreamARN(STREAM_NAME, KINESIS_REGION)).thenReturn(Optional.of(DEFAULT_ARN)); - when(StreamARNUtil.getStreamARN(STREAM_NAME, KINESIS_REGION, Optional.of(TEST_ACCOUNT_ID))) + when(getStreamARN(anyString(), any(Region.class))).thenReturn(Optional.empty()); + when(getStreamARN(STREAM_NAME, KINESIS_REGION)).thenReturn(Optional.of(DEFAULT_ARN)); + when(getStreamARN(STREAM_NAME, KINESIS_REGION, Optional.of(TEST_ACCOUNT_ID))) .thenReturn(Optional.of(DEFAULT_ARN)); } + /** + * Test patterns that should match a serialization regex. + */ + @Test + public void testMultiStreamDeserializationSuccess() { + for (final String pattern : Arrays.asList( + // arn examples + toArn(KINESIS_REGION).toString(), + // serialization examples + "123456789012:stream-name:123", + "123456789012:stream-name:123:" + Region.US_ISOB_EAST_1 + )) { + final StreamIdentifier si = StreamIdentifier.multiStreamInstance(pattern); + assertNotNull(si); + } + } + + /** + * Test patterns that should not match a serialization regex. + */ + @Test + public void testMultiStreamDeserializationFail() { + for (final String pattern : Arrays.asList( + // arn examples + "arn:aws:kinesis::123456789012:stream/stream-name", // missing region + "arn:aws:kinesis:region::stream/stream-name", // missing account id + "arn:aws:kinesis:region:123456789:stream/stream-name", // account id not 12 digits + "arn:aws:kinesis:region:123456789abc:stream/stream-name", // 12char alphanumeric account id + "arn:aws:kinesis:region:123456789012:stream/", // missing stream-name + // serialization examples + ":stream-name:123", // missing account id + "123456789012:stream-name", // missing delimiter before creation epoch + "accountId:stream-name:123", // non-numeric account id +// "123456789:stream-name:123", // account id not 12 digits + "123456789abc:stream-name:123", // 12char alphanumeric account id + "123456789012::123", // missing stream name + "123456789012:stream-name:", // missing creation epoch + "123456789012:stream-name::", // missing creation epoch; ':' for optional region yet missing region + "123456789012:stream-name::us-east-1", // missing creation epoch + "123456789012:stream-name:abc", // non-numeric creation epoch + "123456789012:stream-name:abc:", // non-numeric creation epoch with ':' yet missing region + "123456789012:stream-name:123:", // ':' for optional region yet missing region + "" + )) { + try { + StreamIdentifier.multiStreamInstance(pattern); + Assert.fail(pattern + " should not have created a StreamIdentifier"); + } catch (final IllegalArgumentException iae) { + // expected; ignore + } + } + } + + @Test + public void testInstanceFromArn() { + final Arn arn = toArn(KINESIS_REGION); + final StreamIdentifier single = StreamIdentifier.singleStreamInstance(arn.toString()); + final StreamIdentifier multi = StreamIdentifier.multiStreamInstance(arn.toString()); + + assertEquals(single, multi); + assertEquals(Optional.of(TEST_ACCOUNT_ID), single.accountIdOptional()); + assertEquals(STREAM_NAME, single.streamName()); + assertEquals(Optional.of(arn), single.streamARNOptional()); + } + + @Test(expected = IllegalArgumentException.class) + public void testInstanceWithoutEpochOrArn() { + when(getStreamARN(STREAM_NAME, KINESIS_REGION, Optional.of(TEST_ACCOUNT_ID))) + .thenReturn(Optional.empty()); + + final Arn arn = toArn(KINESIS_REGION); + StreamIdentifier.singleStreamInstance(arn.toString()); + } + @Test public void testSingleStreamInstanceWithName() { StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME); @@ -54,44 +130,64 @@ public class StreamIdentifierTest { StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME, KINESIS_REGION); assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); - assertTrue(actualStreamIdentifier.streamARNOptional().isPresent()); - assertEquals(DEFAULT_ARN, actualStreamIdentifier.streamARNOptional().get()); + assertEquals(Optional.of(DEFAULT_ARN), actualStreamIdentifier.streamARNOptional()); } @Test public void testMultiStreamInstanceWithIdentifierSerialization() { - StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, EPOCH, KINESIS_REGION.toString())); - assertActualStreamIdentifierExpected(DEFAULT_ARN, actualStreamIdentifier); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance(serialize(KINESIS_REGION)); + assertActualStreamIdentifierExpected(actualStreamIdentifier); } @Test public void testMultiStreamInstanceWithRegionSerialized() { Region serializedRegion = Region.US_GOV_EAST_1; - Optional arn = Optional.ofNullable(Arn.builder().partition("aws").service("kinesis") - .accountId(TEST_ACCOUNT_ID).region(serializedRegion.toString()).resource("stream/" + STREAM_NAME).build()); + final Optional arn = Optional.of(toArn(serializedRegion)); - when(StreamARNUtil.getStreamARN(eq(STREAM_NAME), eq(serializedRegion), any(Optional.class))).thenReturn(arn); + when(getStreamARN(STREAM_NAME, serializedRegion, Optional.of(TEST_ACCOUNT_ID))).thenReturn(arn); + + final String expectedSerialization = serialize(serializedRegion); StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, EPOCH, serializedRegion.toString()), KINESIS_REGION); + expectedSerialization, KINESIS_REGION); assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); + assertEquals(expectedSerialization, actualStreamIdentifier.serialize()); + verifyStatic(StreamARNUtil.class); + getStreamARN(STREAM_NAME, serializedRegion, Optional.of(TEST_ACCOUNT_ID)); } @Test public void testMultiStreamInstanceWithoutRegionSerialized() { StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, EPOCH), KINESIS_REGION); - assertActualStreamIdentifierExpected(DEFAULT_ARN, actualStreamIdentifier); + serialize(null), KINESIS_REGION); + assertActualStreamIdentifierExpected(actualStreamIdentifier); } - private void assertActualStreamIdentifierExpected(Optional expectedARN, StreamIdentifier actual) { - assertActualStreamIdentifierExpected(expectedARN.get(), actual); + private void assertActualStreamIdentifierExpected(StreamIdentifier actual) { + assertActualStreamIdentifierExpected(Optional.of(DEFAULT_ARN), actual); } - private void assertActualStreamIdentifierExpected(Arn expectedARN, StreamIdentifier actual) { - assertTrue(actual.streamCreationEpochOptional().isPresent()); - assertTrue(actual.accountIdOptional().isPresent()); - assertTrue(actual.streamARNOptional().isPresent()); - assertEquals(Optional.of(expectedARN), actual.streamARNOptional()); + private void assertActualStreamIdentifierExpected(Optional expectedArn, StreamIdentifier actual) { + assertEquals(STREAM_NAME, actual.streamName()); + assertEquals(Optional.of(EPOCH), actual.streamCreationEpochOptional()); + assertEquals(Optional.of(TEST_ACCOUNT_ID), actual.accountIdOptional()); + assertEquals(expectedArn, actual.streamARNOptional()); + } + + /** + * Creates a pattern that matches {@link StreamIdentifier} serialization. + * + * @param region (optional) region to serialize + */ + private static String serialize(final Region region) { + return String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, Long.toString(EPOCH)) + + ((region == null) ? "" : ':' + region.toString()); + } + + private static Arn toArn(final Region region) { + return Arn.builder().partition("aws").service("kinesis") + .accountId(TEST_ACCOUNT_ID) + .resource("stream/" + STREAM_NAME) + .region(region.toString()) + .build(); } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java index 9f8b34ac..f7492d8d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java @@ -73,7 +73,7 @@ public class PeriodicShardSyncManagerTest { @Before public void setup() { - streamIdentifier = StreamIdentifier.multiStreamInstance("123:stream:456"); + streamIdentifier = StreamIdentifier.multiStreamInstance("123456789012:stream:456"); periodicShardSyncManager = new PeriodicShardSyncManager("worker", leaderDecider, leaseRefresher, currentStreamConfigMap, shardSyncTaskManagerProvider, true, new NullMetricsFactory(), 2 * 60 * 1000, 3); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index aeb07e66..1be0a9d4 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -456,7 +456,7 @@ public class SchedulerTest { .shardId("some_random_shard_id")) .collect(Collectors.toCollection(LinkedList::new)); // Include a stream that is already tracked by multiStreamTracker, just to make sure we will not touch this stream config later - leasesInTable.add(new MultiStreamLease().streamIdentifier("acc1:stream1:1").shardId("some_random_shard_id")); + leasesInTable.add(new MultiStreamLease().streamIdentifier("123456789012:stream1:1").shardId("some_random_shard_id")); // Expected StreamConfig after running syncStreamsFromLeaseTableOnAppInit // By default, Stream not present in multiStreamTracker will have initial position of LATEST @@ -489,7 +489,7 @@ public class SchedulerTest { .shardId("some_random_shard_id")) .collect(Collectors.toCollection(LinkedList::new)); // Include a stream that is already tracked by multiStreamTracker, just to make sure we will not touch this stream config later - leasesInTable.add(new MultiStreamLease().streamIdentifier("acc1:stream1:1").shardId("some_random_shard_id")); + leasesInTable.add(new MultiStreamLease().streamIdentifier("123456789012:stream1:1").shardId("some_random_shard_id")); // Expected StreamConfig after running syncStreamsFromLeaseTableOnAppInit // Stream not present in multiStreamTracker will have initial position specified by orphanedStreamInitialPositionInStream @@ -1299,13 +1299,13 @@ public class SchedulerTest { @Override public List streamConfigList(){ return new ArrayList() {{ - add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( + add(new StreamConfig(StreamIdentifier.multiStreamInstance("123456789012:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( InitialPositionInStream.LATEST))); - add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream2:2"), InitialPositionInStreamExtended.newInitialPosition( + add(new StreamConfig(StreamIdentifier.multiStreamInstance("123456789012:stream2:2"), InitialPositionInStreamExtended.newInitialPosition( InitialPositionInStream.LATEST))); - add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( + add(new StreamConfig(StreamIdentifier.multiStreamInstance("210987654321:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( InitialPositionInStream.LATEST))); - add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream2:3"), InitialPositionInStreamExtended.newInitialPosition( + add(new StreamConfig(StreamIdentifier.multiStreamInstance("210987654321:stream2:3"), InitialPositionInStreamExtended.newInitialPosition( InitialPositionInStream.LATEST))); }}; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index d9f36481..9e130c38 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -92,7 +92,7 @@ public class HierarchicalShardSyncerTest { private static final String LEASE_OWNER = "TestOwner"; private static final MetricsScope SCOPE = new NullMetricsScope(); private static final boolean MULTISTREAM_MODE_ON = true; - private static final String STREAM_IDENTIFIER = "acc:stream:1"; + private static final String STREAM_IDENTIFIER = "123456789012:stream:1"; private static final HierarchicalShardSyncer.MultiStreamArgs MULTI_STREAM_ARGS = new HierarchicalShardSyncer.MultiStreamArgs( MULTISTREAM_MODE_ON, StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java new file mode 100644 index 00000000..7f978bdc --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java @@ -0,0 +1,66 @@ +package software.amazon.kinesis.retrieval; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static software.amazon.kinesis.retrieval.KinesisClientFacade.describeStreamSummary; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; +import software.amazon.awssdk.services.kinesis.model.LimitExceededException; + +@RunWith(MockitoJUnitRunner.class) +public class KinesisClientFacadeTest { + + @Mock + private KinesisAsyncClient mockKinesisClient; + + @Before + public void setUp() { + KinesisClientFacade.initialize(mockKinesisClient); + } + + @Test + public void testDescribeStreamSummary() { + final DescribeStreamSummaryResponse expectedResponse = DescribeStreamSummaryResponse.builder().build(); + when(mockKinesisClient.describeStreamSummary(any(DescribeStreamSummaryRequest.class))) + .thenReturn(CompletableFuture.completedFuture(expectedResponse)); + + final DescribeStreamSummaryResponse actualResponse = describeStreamSummary("narf"); + assertEquals(expectedResponse, actualResponse); + + verify(mockKinesisClient).describeStreamSummary(any(DescribeStreamSummaryRequest.class)); + } + + @Test + public void testDescribeStreamSummaryRetries() throws Exception { + final DescribeStreamSummaryResponse expectedResponse = DescribeStreamSummaryResponse.builder().build(); + final CompletableFuture mockFuture = mock(CompletableFuture.class); + final ExecutionException executionException = new ExecutionException(LimitExceededException.builder().build()); + + when(mockKinesisClient.describeStreamSummary(any(DescribeStreamSummaryRequest.class))) + .thenReturn(mockFuture); + when(mockFuture.get()) + .thenThrow(executionException) + .thenThrow(executionException) + .thenReturn(expectedResponse); + + final DescribeStreamSummaryResponse actualResponse = describeStreamSummary("retry me plz"); + assertEquals(expectedResponse, actualResponse); + + verify(mockKinesisClient, times(3)).describeStreamSummary(any(DescribeStreamSummaryRequest.class)); + verify(mockFuture, times(3)).get(); + } +} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java index 32ca17ce..58454087 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java @@ -94,14 +94,14 @@ public class FanOutConfigTest { public void testRegisterNotCalledWhenConsumerArnSetInMultiStreamMode() throws Exception { when(streamConfig.consumerArn()).thenReturn("consumerArn"); - getRecordsCache("account:stream:12345"); + getRecordsCache("123456789012:stream:12345"); verify(consumerRegistration, never()).getOrCreateStreamConsumerArn(); } @Test public void testRegisterCalledWhenConsumerArnNotSetInMultiStreamMode() throws Exception { - getRecordsCache("account:stream:12345"); + getRecordsCache("123456789012:stream:12345"); verify(consumerRegistration).getOrCreateStreamConsumerArn(); } From e3d845a1f517fbe2c6760aa7740a103668075c0f Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Thu, 20 Apr 2023 19:18:14 -0400 Subject: [PATCH 56/88] StreamARN: removed region from `StreamIdentifier` serialization. (#1099) Provided ARNs must share the same region as the Kinesis endpoint. --- .../amazon/kinesis/common/StreamARNUtil.java | 18 ++-- .../kinesis/common/StreamIdentifier.java | 46 +++++---- .../kinesis/common/StreamARNUtilTest.java | 41 ++++---- .../kinesis/common/StreamIdentifierTest.java | 98 ++++++++----------- 4 files changed, 102 insertions(+), 101 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java index 03652a5b..667bf820 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java @@ -2,7 +2,6 @@ package software.amazon.kinesis.common; import lombok.AccessLevel; import lombok.NoArgsConstructor; -import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.awscore.exception.AwsServiceException; @@ -26,7 +25,11 @@ public final class StreamARNUtil { try (final SdkHttpClient httpClient = UrlConnectionHttpClient.builder().build(); final StsClient stsClient = StsClient.builder().httpClient(httpClient).build()) { final GetCallerIdentityResponse response = stsClient.getCallerIdentity(); - return Arn.fromString(response.arn()); + final Arn arn = Arn.fromString(response.arn()); + + // guarantee the cached ARN will never have an empty accountId + arn.accountId().orElseThrow(() -> new IllegalStateException("AccountId is not present on " + arn)); + return arn; } catch (AwsServiceException | SdkClientException e) { log.warn("Unable to get sts caller identity to build stream arn", e); return null; @@ -34,17 +37,18 @@ public final class StreamARNUtil { }); /** - * This static method attempts to retrieve the stream ARN using the stream name, region, and accountId returned by STS + * Retrieves the stream ARN using the stream name, region, and accountId returned by STS. * It is designed to fail gracefully, returning Optional.empty() if any errors occur. * * @param streamName stream name - * @param kinesisRegion kinesisRegion is a nullable parameter used to construct the stream arn + * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be + * processed are located. A null guarantees an empty ARN. */ public static Optional getStreamARN(String streamName, Region kinesisRegion) { - return getStreamARN(streamName, kinesisRegion, Optional.empty()); + return getStreamARN(streamName, kinesisRegion, null); } - public static Optional getStreamARN(String streamName, Region kinesisRegion, @NonNull Optional accountId) { + public static Optional getStreamARN(String streamName, Region kinesisRegion, String accountId) { if (kinesisRegion == null) { return Optional.empty(); } @@ -55,7 +59,7 @@ public final class StreamARNUtil { } // the provided accountId takes precedence - final String chosenAccountId = accountId.orElse(identityArn.accountId().orElse("")); + final String chosenAccountId = (accountId != null) ? accountId : identityArn.accountId().get(); return Optional.of(Arn.builder() .partition(identityArn.partition()) .service("kinesis") diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java index 1a81f606..60195d4f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -48,14 +48,10 @@ public class StreamIdentifier { /** * Pattern for a serialized {@link StreamIdentifier}. The valid format is - * {@code ::[:]} where - * {@code region} is the id representation of a {@link Region} and is - * optional. + * {@code ::}. */ private static final Pattern STREAM_IDENTIFIER_PATTERN = Pattern.compile( - // `?::` has two parts: `?:` starts a non-capturing group, and - // `:` is the first character in the group (i.e., ":") - "(?[0-9]+):(?[^:]+):(?[0-9]+)(?::(?[-a-z0-9]+))?"); + "(?[0-9]+):(?[^:]+):(?[0-9]+)"); /** * Pattern for a stream ARN. The valid format is @@ -96,7 +92,6 @@ public class StreamIdentifier { final StringBuilder sb = new StringBuilder(accountIdOptional.get()).append(delimiter) .append(streamName).append(delimiter); streamCreationEpochOptional.ifPresent(sb::append); - streamARNOptional.flatMap(Arn::region).ifPresent(region -> sb.append(delimiter).append(region)); return sb.toString(); } @@ -121,8 +116,10 @@ public class StreamIdentifier { * Create a multi stream instance for StreamIdentifier from serialized stream identifier. * * @param serializationOrArn serialized {@link StreamIdentifier} or AWS ARN of a Kinesis stream - * @param kinesisRegion This nullable region is used to construct the optional StreamARN + * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be + * processed are located. A null will default to the caller's region. * + * @see #multiStreamInstance(String) * @see #serialize() */ public static StreamIdentifier multiStreamInstance(String serializationOrArn, Region kinesisRegion) { @@ -142,6 +139,8 @@ public class StreamIdentifier { * Create a single stream instance for StreamIdentifier from stream name. * * @param streamNameOrArn stream name or AWS ARN of a Kinesis stream + * + * @see #singleStreamInstance(String, Region) */ public static StreamIdentifier singleStreamInstance(String streamNameOrArn) { return singleStreamInstance(streamNameOrArn, null); @@ -152,7 +151,10 @@ public class StreamIdentifier { * This method also constructs the optional StreamARN based on the region info. * * @param streamNameOrArn stream name or AWS ARN of a Kinesis stream - * @param kinesisRegion (optional) region used to construct the ARN + * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be + * processed are located. A null will default to the caller's region. + * + * @see #singleStreamInstance(String) */ public static StreamIdentifier singleStreamInstance(String streamNameOrArn, Region kinesisRegion) { Validate.notEmpty(streamNameOrArn, "StreamName should not be empty"); @@ -172,7 +174,8 @@ public class StreamIdentifier { * Deserializes a StreamIdentifier from {@link #STREAM_IDENTIFIER_PATTERN}. * * @param input input string (e.g., ARN, serialized instance) to convert into an instance - * @param kinesisRegion (optional) region used to construct the ARN + * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be + * processed are located. A null will default to the caller's region. * @return a StreamIdentifier instance if the pattern matched, otherwise null */ private static StreamIdentifier fromSerialization(final String input, final Region kinesisRegion) { @@ -185,31 +188,38 @@ public class StreamIdentifier { * Constructs a StreamIdentifier from {@link #STREAM_ARN_PATTERN}. * * @param input input string (e.g., ARN, serialized instance) to convert into an instance - * @param kinesisRegion (optional) region used to construct the ARN + * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be + * processed are located. A null will default to the caller's region. * @return a StreamIdentifier instance if the pattern matched, otherwise null */ private static StreamIdentifier fromArn(final String input, final Region kinesisRegion) { final Matcher matcher = STREAM_ARN_PATTERN.matcher(input); - return matcher.matches() - ? toStreamIdentifier(matcher, "", kinesisRegion) : null; + if (matcher.matches()) { + final String arnRegion = matcher.group("region"); + final Region region = (arnRegion != null) ? Region.of(arnRegion) : kinesisRegion; + if ((kinesisRegion != null) && (region != kinesisRegion)) { + throw new IllegalArgumentException(String.format( + "Cannot create StreamIdentifier for a region other than %s: %s", kinesisRegion, input)); + } + return toStreamIdentifier(matcher, "", region); + } + return null; } private static StreamIdentifier toStreamIdentifier(final Matcher matcher, final String matchedEpoch, final Region kinesisRegion) { - final Optional accountId = Optional.of(matcher.group("accountId")); + final String accountId = matcher.group("accountId"); final String streamName = matcher.group("streamName"); final Optional creationEpoch = matchedEpoch.isEmpty() ? Optional.empty() : Optional.of(Long.valueOf(matchedEpoch)); - final String matchedRegion = matcher.group("region"); - final Region region = (matchedRegion != null) ? Region.of(matchedRegion) : kinesisRegion; - final Optional arn = StreamARNUtil.getStreamARN(streamName, region, accountId); + final Optional arn = StreamARNUtil.getStreamARN(streamName, kinesisRegion, accountId); if (!creationEpoch.isPresent() && !arn.isPresent()) { throw new IllegalArgumentException("Cannot create StreamIdentifier if missing both ARN and creation epoch"); } return StreamIdentifier.builder() - .accountIdOptional(accountId) + .accountIdOptional(Optional.of(accountId)) .streamName(streamName) .streamCreationEpochOptional(creationEpoch) .streamARNOptional(arn) diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java index 595710c6..aef3974b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java @@ -17,6 +17,7 @@ import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sts.StsClient; import software.amazon.awssdk.services.sts.StsClientBuilder; +import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; import java.lang.reflect.Field; import java.util.Optional; @@ -60,7 +61,8 @@ public class StreamARNUtilTest { public void setUp() throws Exception { MockitoAnnotations.initMocks(this); - setUpSupplierCache(); + spySupplierCache = spy(ORIGINAL_CACHE); + setUpSupplierCache(spySupplierCache); final Arn defaultArn = toArn(STS_RESPONSE_ARN_FORMAT, ACCOUNT_ID); doReturn(defaultArn).when(spySupplierCache).get(); @@ -84,12 +86,10 @@ public class StreamARNUtilTest { * one-and-done cache behavior, provide each test precise control over * return values, and enable the ability to verify interactions via Mockito. */ - private void setUpSupplierCache() throws Exception { - spySupplierCache = spy(ORIGINAL_CACHE); - + static void setUpSupplierCache(final SupplierCache cache) throws Exception { final Field f = StreamARNUtil.class.getDeclaredField("CALLER_IDENTITY_ARN"); f.setAccessible(true); - f.set(null, spySupplierCache); + f.set(null, cache); f.setAccessible(false); } @@ -124,6 +124,24 @@ public class StreamARNUtilTest { verify(spySupplierCache, times(2)).get(); } + @Test(expected = IllegalStateException.class) + public void testStsResponseWithoutAccountId() { + setUpSts(); + + final Arn arnWithoutAccountId = toArn(STS_RESPONSE_ARN_FORMAT, ""); + assertEquals(Optional.empty(), arnWithoutAccountId.accountId()); + + final GetCallerIdentityResponse identityResponse = GetCallerIdentityResponse.builder() + .arn(arnWithoutAccountId.toString()).build(); + when(mockStsClient.getCallerIdentity()).thenReturn(identityResponse); + + try { + StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); + } finally { + verify(mockStsClient).getCallerIdentity(); + } + } + @Test public void testGetStreamARNReturnsEmptyOnInvalidKinesisRegion() { // Optional.empty() is expected when kinesis region is not set correctly @@ -143,7 +161,7 @@ public class StreamARNUtilTest { when(spySupplierCache.get()).thenReturn(cachedArn); final Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1, - Optional.of(providedAccountId)); + providedAccountId); final Arn expectedStreamARN = toArn(KINESIS_STREAM_ARN_FORMAT, providedAccountId, STREAM_NAME); verify(spySupplierCache).get(); @@ -152,17 +170,6 @@ public class StreamARNUtilTest { assertEquals(expectedStreamARN, actualStreamARNOptional.get()); } - @Test - public void testNoAccountId() { - final Arn arnWithoutAccountId = toArn(STS_RESPONSE_ARN_FORMAT, ""); - when(spySupplierCache.get()).thenReturn(arnWithoutAccountId); - assertEquals(Optional.empty(), arnWithoutAccountId.accountId()); - - final Optional actualArn = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); - assertTrue(actualArn.isPresent()); - assertEquals(Optional.empty(), actualArn.get().accountId()); - } - private static Optional getStreamArn() { final Optional actualArn = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); final Arn expectedArn = toArn(KINESIS_STREAM_ARN_FORMAT, ACCOUNT_ID, STREAM_NAME); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java index 115cab03..b3f4991b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java @@ -1,7 +1,7 @@ package software.amazon.kinesis.common; import org.junit.Assert; -import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -14,9 +14,7 @@ import java.util.Optional; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyString; +import static org.junit.Assert.assertNotEquals; import static org.mockito.Mockito.when; import static org.powermock.api.mockito.PowerMockito.mockStatic; import static org.powermock.api.mockito.PowerMockito.verifyStatic; @@ -32,14 +30,9 @@ public class StreamIdentifierTest { private static final Arn DEFAULT_ARN = toArn(KINESIS_REGION); - @Before - public void setUp() { - mockStatic(StreamARNUtil.class); - - when(getStreamARN(anyString(), any(Region.class))).thenReturn(Optional.empty()); - when(getStreamARN(STREAM_NAME, KINESIS_REGION)).thenReturn(Optional.of(DEFAULT_ARN)); - when(getStreamARN(STREAM_NAME, KINESIS_REGION, Optional.of(TEST_ACCOUNT_ID))) - .thenReturn(Optional.of(DEFAULT_ARN)); + @BeforeClass + public static void setUpBeforeClass() throws Exception { + StreamARNUtilTest.setUpSupplierCache(new SupplierCache<>(() -> DEFAULT_ARN)); } /** @@ -47,16 +40,9 @@ public class StreamIdentifierTest { */ @Test public void testMultiStreamDeserializationSuccess() { - for (final String pattern : Arrays.asList( - // arn examples - toArn(KINESIS_REGION).toString(), - // serialization examples - "123456789012:stream-name:123", - "123456789012:stream-name:123:" + Region.US_ISOB_EAST_1 - )) { - final StreamIdentifier si = StreamIdentifier.multiStreamInstance(pattern); - assertNotNull(si); - } + final StreamIdentifier siSerialized = StreamIdentifier.multiStreamInstance(serialize()); + assertEquals(Optional.of(EPOCH), siSerialized.streamCreationEpochOptional()); + assertActualStreamIdentifierExpected(null, siSerialized); } /** @@ -73,17 +59,12 @@ public class StreamIdentifierTest { "arn:aws:kinesis:region:123456789012:stream/", // missing stream-name // serialization examples ":stream-name:123", // missing account id - "123456789012:stream-name", // missing delimiter before creation epoch - "accountId:stream-name:123", // non-numeric account id // "123456789:stream-name:123", // account id not 12 digits "123456789abc:stream-name:123", // 12char alphanumeric account id "123456789012::123", // missing stream name + "123456789012:stream-name", // missing delimiter and creation epoch "123456789012:stream-name:", // missing creation epoch - "123456789012:stream-name::", // missing creation epoch; ':' for optional region yet missing region - "123456789012:stream-name::us-east-1", // missing creation epoch "123456789012:stream-name:abc", // non-numeric creation epoch - "123456789012:stream-name:abc:", // non-numeric creation epoch with ':' yet missing region - "123456789012:stream-name:123:", // ':' for optional region yet missing region "" )) { try { @@ -102,18 +83,22 @@ public class StreamIdentifierTest { final StreamIdentifier multi = StreamIdentifier.multiStreamInstance(arn.toString()); assertEquals(single, multi); - assertEquals(Optional.of(TEST_ACCOUNT_ID), single.accountIdOptional()); - assertEquals(STREAM_NAME, single.streamName()); - assertEquals(Optional.of(arn), single.streamARNOptional()); + assertEquals(Optional.empty(), single.streamCreationEpochOptional()); + assertActualStreamIdentifierExpected(arn, single); } @Test(expected = IllegalArgumentException.class) public void testInstanceWithoutEpochOrArn() { - when(getStreamARN(STREAM_NAME, KINESIS_REGION, Optional.of(TEST_ACCOUNT_ID))) + mockStatic(StreamARNUtil.class); + when(getStreamARN(STREAM_NAME, KINESIS_REGION, TEST_ACCOUNT_ID)) .thenReturn(Optional.empty()); - final Arn arn = toArn(KINESIS_REGION); - StreamIdentifier.singleStreamInstance(arn.toString()); + try { + StreamIdentifier.singleStreamInstance(DEFAULT_ARN.toString()); + } finally { + verifyStatic(StreamARNUtil.class); + getStreamARN(STREAM_NAME, KINESIS_REGION, TEST_ACCOUNT_ID); + } } @Test @@ -130,57 +115,52 @@ public class StreamIdentifierTest { StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME, KINESIS_REGION); assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); + assertEquals(STREAM_NAME, actualStreamIdentifier.streamName()); assertEquals(Optional.of(DEFAULT_ARN), actualStreamIdentifier.streamARNOptional()); } @Test public void testMultiStreamInstanceWithIdentifierSerialization() { - StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance(serialize(KINESIS_REGION)); - assertActualStreamIdentifierExpected(actualStreamIdentifier); + StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance(serialize()); + assertActualStreamIdentifierExpected(null, actualStreamIdentifier); + assertEquals(Optional.of(EPOCH), actualStreamIdentifier.streamCreationEpochOptional()); } - @Test - public void testMultiStreamInstanceWithRegionSerialized() { - Region serializedRegion = Region.US_GOV_EAST_1; - final Optional arn = Optional.of(toArn(serializedRegion)); + /** + * When KCL's Kinesis endpoint is a region, it lacks visibility to streams + * in other regions. Therefore, when the endpoint and ARN conflict, an + * Exception should be thrown. + */ + @Test(expected = IllegalArgumentException.class) + public void testConflictOnRegions() { + final Region arnRegion = Region.US_GOV_EAST_1; + assertNotEquals(arnRegion, KINESIS_REGION); - when(getStreamARN(STREAM_NAME, serializedRegion, Optional.of(TEST_ACCOUNT_ID))).thenReturn(arn); - - final String expectedSerialization = serialize(serializedRegion); - StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - expectedSerialization, KINESIS_REGION); - assertActualStreamIdentifierExpected(arn, actualStreamIdentifier); - assertEquals(expectedSerialization, actualStreamIdentifier.serialize()); - verifyStatic(StreamARNUtil.class); - getStreamARN(STREAM_NAME, serializedRegion, Optional.of(TEST_ACCOUNT_ID)); + StreamIdentifier.multiStreamInstance(toArn(arnRegion).toString(), KINESIS_REGION); } @Test public void testMultiStreamInstanceWithoutRegionSerialized() { StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - serialize(null), KINESIS_REGION); + serialize(), KINESIS_REGION); assertActualStreamIdentifierExpected(actualStreamIdentifier); } private void assertActualStreamIdentifierExpected(StreamIdentifier actual) { - assertActualStreamIdentifierExpected(Optional.of(DEFAULT_ARN), actual); + assertActualStreamIdentifierExpected(DEFAULT_ARN, actual); } - private void assertActualStreamIdentifierExpected(Optional expectedArn, StreamIdentifier actual) { + private void assertActualStreamIdentifierExpected(Arn expectedArn, StreamIdentifier actual) { assertEquals(STREAM_NAME, actual.streamName()); - assertEquals(Optional.of(EPOCH), actual.streamCreationEpochOptional()); assertEquals(Optional.of(TEST_ACCOUNT_ID), actual.accountIdOptional()); - assertEquals(expectedArn, actual.streamARNOptional()); + assertEquals(Optional.ofNullable(expectedArn), actual.streamARNOptional()); } /** * Creates a pattern that matches {@link StreamIdentifier} serialization. - * - * @param region (optional) region to serialize */ - private static String serialize(final Region region) { - return String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, Long.toString(EPOCH)) + - ((region == null) ? "" : ':' + region.toString()); + private static String serialize() { + return String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, Long.toString(EPOCH)); } private static Arn toArn(final Region region) { From f115235fd1f0d6038d39de9db7c8cdde1ebb1db9 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Fri, 5 May 2023 14:19:27 -0400 Subject: [PATCH 57/88] Refactored `SynchronizedCache` out of `SupplierCache`, and introduced (#1107) `FunctionCache`. --- .../amazon/kinesis/common/FunctionCache.java | 50 +++++++++++++++ .../amazon/kinesis/common/SupplierCache.java | 14 +---- .../kinesis/common/SynchronizedCache.java | 48 ++++++++++++++ .../kinesis/common/FunctionCacheTest.java | 61 ++++++++++++++++++ .../kinesis/common/SynchronizedCacheTest.java | 62 +++++++++++++++++++ 5 files changed, 223 insertions(+), 12 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java new file mode 100644 index 00000000..881cf5a9 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java @@ -0,0 +1,50 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.common; + +import java.util.function.Function; + +import lombok.RequiredArgsConstructor; + +/** + * Caches the result from a {@link Function}. Caching is especially useful when + * invoking the function is an expensive call that produces a reusable result. + * If the input value should be fixed, {@link SupplierCache} may be used. + *

+ * Note that if {@code f(x)=X} is cached, {@code X} will be returned for every + * successive query of this cache regardless of the input parameter. This is + * by design under the assumption that {@code X} is a viable response for + * other invocations. + * + * @param input type + * @param output type + */ +@RequiredArgsConstructor +public class FunctionCache extends SynchronizedCache { + + private final Function function; + + /** + * Returns the cached result. If the cache is null, the function will be + * invoked to populate the cache. + * + * @param input input argument to the underlying function + * @return cached result which may be null + */ + public OUT get(final IN input) { + return get(() -> function.apply(input)); + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java index 632e4b8f..72d75b92 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java @@ -9,12 +9,10 @@ import lombok.RequiredArgsConstructor; * {@link Supplier#get()} is an expensive call that produces static results. */ @RequiredArgsConstructor -public class SupplierCache { +public class SupplierCache extends SynchronizedCache { private final Supplier supplier; - private volatile T result; - /** * Returns the cached result. If the cache is null, the supplier will be * invoked to populate the cache. @@ -22,15 +20,7 @@ public class SupplierCache { * @return cached result which may be null */ public T get() { - if (result == null) { - synchronized (this) { - // double-check lock - if (result == null) { - result = supplier.get(); - } - } - } - return result; + return get(supplier); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java new file mode 100644 index 00000000..3df241d3 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java @@ -0,0 +1,48 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.common; + +import java.util.function.Supplier; + +/** + * A synchronized, "no frills" cache that preserves the first non-null value + * returned from a {@link Supplier}. + * + * @param result type + */ +public class SynchronizedCache { + + private volatile R result; + + /** + * Returns the cached result. If the cache is null, the supplier will be + * invoked to populate the cache. + * + * @param supplier supplier to invoke if the cache is null + * @return cached result which may be null + */ + protected R get(final Supplier supplier) { + if (result == null) { + synchronized (this) { + // double-check lock + if (result == null) { + result = supplier.get(); + } + } + } + return result; + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java new file mode 100644 index 00000000..2f55af4b --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java @@ -0,0 +1,61 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.common; + +import java.util.function.Function; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class FunctionCacheTest { + + @Mock + private Function mockFunction; + + private FunctionCache cache; + + @Before + public void setUp() { + cache = new FunctionCache<>(mockFunction); + } + + /** + * Test that the cache stops invoking the encapsulated {@link Function} + * after it returns a non-null value. + */ + @Test + public void testCache() { + final int expectedValue = 3; + when(mockFunction.apply(expectedValue)).thenReturn(expectedValue); + + assertNull(cache.get(1)); + assertNull(cache.get(2)); + assertEquals(expectedValue, cache.get(3)); + assertEquals(expectedValue, cache.get(4)); + assertEquals(expectedValue, cache.get(5)); + verify(mockFunction, times(expectedValue)).apply(anyInt()); + } +} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java new file mode 100644 index 00000000..bad3f3cf --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java @@ -0,0 +1,62 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.common; + +import java.util.function.Supplier; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class SynchronizedCacheTest { + + private static final Object DUMMY_RESULT = SynchronizedCacheTest.class; + + @Mock + private Supplier mockSupplier; + + private final SynchronizedCache cache = new SynchronizedCache<>(); + + @Test + public void testCache() { + when(mockSupplier.get()).thenReturn(DUMMY_RESULT); + + final Object result1 = cache.get(mockSupplier); + final Object result2 = cache.get(mockSupplier); + + assertEquals(DUMMY_RESULT, result1); + assertSame(result1, result2); + verify(mockSupplier).get(); + } + + @Test + public void testCacheWithNullResult() { + when(mockSupplier.get()).thenReturn(null).thenReturn(DUMMY_RESULT); + + assertNull(cache.get(mockSupplier)); + assertEquals(DUMMY_RESULT, cache.get(mockSupplier)); + assertEquals(DUMMY_RESULT, cache.get(mockSupplier)); + verify(mockSupplier, times(2)).get(); + } +} \ No newline at end of file From ee3f56ae660ae86a4d48c4953eb48afb7e2628b5 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Fri, 5 May 2023 14:29:34 -0400 Subject: [PATCH 58/88] Added missing copyright to new files. (#1106) --- .../amazon/kinesis/common/StreamARNUtil.java | 14 ++++++++++++++ .../amazon/kinesis/common/SupplierCache.java | 16 ++++++++++++++++ .../kinesis/retrieval/KinesisClientFacade.java | 14 ++++++++++++++ .../amazon/kinesis/common/StreamARNUtilTest.java | 14 ++++++++++++++ .../amazon/kinesis/common/SupplierCacheTest.java | 14 ++++++++++++++ .../retrieval/KinesisClientFacadeTest.java | 14 ++++++++++++++ 6 files changed, 86 insertions(+) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java index 667bf820..89cf6331 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java @@ -1,3 +1,17 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.common; import lombok.AccessLevel; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java index 72d75b92..abe2822d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java @@ -1,3 +1,17 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.common; import java.util.function.Supplier; @@ -7,6 +21,8 @@ import lombok.RequiredArgsConstructor; /** * Caches results from a {@link Supplier}. Caching is especially useful when * {@link Supplier#get()} is an expensive call that produces static results. + * + * @param result type */ @RequiredArgsConstructor public class SupplierCache extends SynchronizedCache { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java index 5c9e732b..d2f90d3a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java @@ -1,3 +1,17 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.retrieval; import java.util.concurrent.ExecutionException; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java index aef3974b..b6009b43 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java @@ -1,3 +1,17 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.common; import org.junit.Before; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java index 4df4f81d..a0bde098 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java @@ -1,3 +1,17 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.common; import java.util.function.Supplier; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java index 7f978bdc..c2095d87 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java @@ -1,3 +1,17 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.retrieval; import java.util.concurrent.CompletableFuture; From 53c76c72c3100e437bf656ad8afac44d4db26bb6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 15 May 2023 14:16:37 -0700 Subject: [PATCH 59/88] Bump maven-compiler-plugin from 3.8.1 to 3.11.0 (#1071) Bumps [maven-compiler-plugin](https://github.com/apache/maven-compiler-plugin) from 3.8.1 to 3.11.0. - [Release notes](https://github.com/apache/maven-compiler-plugin/releases) - [Commits](https://github.com/apache/maven-compiler-plugin/compare/maven-compiler-plugin-3.8.1...maven-compiler-plugin-3.11.0) --- updated-dependencies: - dependency-name: org.apache.maven.plugins:maven-compiler-plugin dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index dd20862e..db364375 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -130,7 +130,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.8.1 + 3.11.0 1.8 1.8 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 7356a72b..4c0bc5c6 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -202,7 +202,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.8.1 + 3.11.0 1.8 1.8 diff --git a/pom.xml b/pom.xml index fc3ad7a2..e4ee6fc7 100644 --- a/pom.xml +++ b/pom.xml @@ -77,7 +77,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.8.1 + 3.11.0 1.8 1.8 From 58ceaf45263655a11f8cb57951fc26f458aaa103 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 16 May 2023 14:29:20 -0700 Subject: [PATCH 60/88] Bump awssdk.version from 2.20.40 to 2.20.43 (#1090) Bumps `awssdk.version` from 2.20.40 to 2.20.43. Updates `kinesis` from 2.20.40 to 2.20.43 Updates `dynamodb` from 2.20.40 to 2.20.43 Updates `cloudwatch` from 2.20.40 to 2.20.43 Updates `netty-nio-client` from 2.20.40 to 2.20.43 Updates `sts` from 2.20.40 to 2.20.43 --- updated-dependencies: - dependency-name: software.amazon.awssdk:kinesis dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:dynamodb dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:cloudwatch dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:netty-nio-client dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: software.amazon.awssdk:sts dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index e4ee6fc7..0611f950 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.20.40 + 2.20.43 From 32a29dbf49771a7e6ff3dce666820afe9a62bb6e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 16 May 2023 14:36:00 -0700 Subject: [PATCH 61/88] Bump slf4j-api from 2.0.6 to 2.0.7 (#1073) Bumps [slf4j-api](https://github.com/qos-ch/slf4j) from 2.0.6 to 2.0.7. - [Release notes](https://github.com/qos-ch/slf4j/releases) - [Commits](https://github.com/qos-ch/slf4j/commits) --- updated-dependencies: - dependency-name: org.slf4j:slf4j-api dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 4c0bc5c6..5560c719 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -50,7 +50,7 @@ 1.0.392 libsqlite4java ${project.build.directory}/test-lib - 2.0.6 + 2.0.7 1.1.14 From 7092ffdbd6a576adca9c513998734e3ec844c70f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 16 May 2023 15:32:01 -0700 Subject: [PATCH 62/88] Bump nexus-staging-maven-plugin from 1.6.8 to 1.6.13 (#1072) Bumps nexus-staging-maven-plugin from 1.6.8 to 1.6.13. --- updated-dependencies: - dependency-name: org.sonatype.plugins:nexus-staging-maven-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0611f950..0f3e0ddf 100644 --- a/pom.xml +++ b/pom.xml @@ -110,7 +110,7 @@ org.sonatype.plugins nexus-staging-maven-plugin - 1.6.8 + 1.6.13 true sonatype-nexus-staging From d7f3a079e1ab01e9a25e5b44bcc66f06e9ebaf7d Mon Sep 17 00:00:00 2001 From: furq-aws <127275086+furq-aws@users.noreply.github.com> Date: Fri, 19 May 2023 12:21:20 -0700 Subject: [PATCH 63/88] Add support for stream ARNs (#1109) Add support for referencing streams by streamARN in single-stream mode, or by the combination of streamARN and creationEpoch in multi-stream mode. --- amazon-kinesis-client/pom.xml | 19 -- .../amazon/kinesis/common/ConfigsBuilder.java | 29 ++- .../amazon/kinesis/common/FunctionCache.java | 50 ---- .../amazon/kinesis/common/StreamARNUtil.java | 86 ------- .../kinesis/common/StreamIdentifier.java | 218 +++++++----------- .../amazon/kinesis/common/SupplierCache.java | 42 ---- .../kinesis/common/SynchronizedCache.java | 48 ---- .../kinesis/leases/KinesisShardDetector.java | 5 +- .../processor/SingleStreamTracker.java | 6 +- .../retrieval/KinesisClientFacade.java | 114 --------- .../kinesis/retrieval/RetrievalConfig.java | 12 +- .../retrieval/polling/KinesisDataFetcher.java | 4 +- .../kinesis/common/ConfigsBuilderTest.java | 38 ++- .../kinesis/common/FunctionCacheTest.java | 61 ----- .../kinesis/common/StreamARNUtilTest.java | 201 ---------------- .../kinesis/common/StreamIdentifierTest.java | 154 ++++++------- .../kinesis/common/SupplierCacheTest.java | 70 ------ .../kinesis/common/SynchronizedCacheTest.java | 62 ----- .../kinesis/coordinator/SchedulerTest.java | 3 +- .../retrieval/KinesisClientFacadeTest.java | 80 ------- .../retrieval/RetrievalConfigTest.java | 36 ++- .../kinesis/utils/MockObjectHelper.java | 31 --- 22 files changed, 254 insertions(+), 1115 deletions(-) delete mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java delete mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java delete mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java delete mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java delete mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java delete mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java delete mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java delete mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java delete mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java delete mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java delete mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 5560c719..e08ed98d 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -75,11 +75,6 @@ netty-nio-client ${awssdk.version} - - software.amazon.awssdk - sts - ${awssdk.version} - software.amazon.glue schema-registry-serde @@ -139,20 +134,6 @@ test - - org.powermock - powermock-module-junit4 - 1.7.4 - test - - - - org.powermock - powermock-api-mockito - 1.7.4 - test - - org.hamcrest hamcrest-all diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java index 57de9059..02258950 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java @@ -25,6 +25,7 @@ import org.apache.commons.lang3.StringUtils; import lombok.NonNull; import lombok.experimental.Accessors; +import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -128,7 +129,7 @@ public class ConfigsBuilder { } /** - * Constructor to initialize ConfigsBuilder for a single stream. + * Constructor to initialize ConfigsBuilder for a single stream identified by name. * * @param streamName * @param applicationName @@ -142,7 +143,31 @@ public class ConfigsBuilder { @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { - this(new SingleStreamTracker(streamName, kinesisClient.serviceClientConfiguration().region()), + this(new SingleStreamTracker(streamName), + applicationName, + kinesisClient, + dynamoDBClient, + cloudWatchClient, + workerIdentifier, + shardRecordProcessorFactory); + } + + /** + * Constructor to initialize ConfigsBuilder for a single stream identified by {@link Arn}. + * + * @param streamArn + * @param applicationName + * @param kinesisClient + * @param dynamoDBClient + * @param cloudWatchClient + * @param workerIdentifier + * @param shardRecordProcessorFactory + */ + public ConfigsBuilder(@NonNull Arn streamArn, @NonNull String applicationName, + @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, + @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, + @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + this(new SingleStreamTracker(streamArn), applicationName, kinesisClient, dynamoDBClient, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java deleted file mode 100644 index 881cf5a9..00000000 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/FunctionCache.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import java.util.function.Function; - -import lombok.RequiredArgsConstructor; - -/** - * Caches the result from a {@link Function}. Caching is especially useful when - * invoking the function is an expensive call that produces a reusable result. - * If the input value should be fixed, {@link SupplierCache} may be used. - *

- * Note that if {@code f(x)=X} is cached, {@code X} will be returned for every - * successive query of this cache regardless of the input parameter. This is - * by design under the assumption that {@code X} is a viable response for - * other invocations. - * - * @param input type - * @param output type - */ -@RequiredArgsConstructor -public class FunctionCache extends SynchronizedCache { - - private final Function function; - - /** - * Returns the cached result. If the cache is null, the function will be - * invoked to populate the cache. - * - * @param input input argument to the underlying function - * @return cached result which may be null - */ - public OUT get(final IN input) { - return get(() -> function.apply(input)); - } - -} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java deleted file mode 100644 index 89cf6331..00000000 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamARNUtil.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import lombok.AccessLevel; -import lombok.NoArgsConstructor; -import lombok.extern.slf4j.Slf4j; -import software.amazon.awssdk.arns.Arn; -import software.amazon.awssdk.awscore.exception.AwsServiceException; -import software.amazon.awssdk.core.exception.SdkClientException; -import software.amazon.awssdk.http.SdkHttpClient; -import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.sts.StsClient; -import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; - -import java.util.Optional; - -@Slf4j -@NoArgsConstructor(access = AccessLevel.PRIVATE) -public final class StreamARNUtil { - - /** - * Caches an {@link Arn} constructed from a {@link StsClient#getCallerIdentity()} call. - */ - private static final SupplierCache CALLER_IDENTITY_ARN = new SupplierCache<>(() -> { - try (final SdkHttpClient httpClient = UrlConnectionHttpClient.builder().build(); - final StsClient stsClient = StsClient.builder().httpClient(httpClient).build()) { - final GetCallerIdentityResponse response = stsClient.getCallerIdentity(); - final Arn arn = Arn.fromString(response.arn()); - - // guarantee the cached ARN will never have an empty accountId - arn.accountId().orElseThrow(() -> new IllegalStateException("AccountId is not present on " + arn)); - return arn; - } catch (AwsServiceException | SdkClientException e) { - log.warn("Unable to get sts caller identity to build stream arn", e); - return null; - } - }); - - /** - * Retrieves the stream ARN using the stream name, region, and accountId returned by STS. - * It is designed to fail gracefully, returning Optional.empty() if any errors occur. - * - * @param streamName stream name - * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be - * processed are located. A null guarantees an empty ARN. - */ - public static Optional getStreamARN(String streamName, Region kinesisRegion) { - return getStreamARN(streamName, kinesisRegion, null); - } - - public static Optional getStreamARN(String streamName, Region kinesisRegion, String accountId) { - if (kinesisRegion == null) { - return Optional.empty(); - } - - final Arn identityArn = CALLER_IDENTITY_ARN.get(); - if (identityArn == null) { - return Optional.empty(); - } - - // the provided accountId takes precedence - final String chosenAccountId = (accountId != null) ? accountId : identityArn.accountId().get(); - return Optional.of(Arn.builder() - .partition(identityArn.partition()) - .service("kinesis") - .region(kinesisRegion.toString()) - .accountId(chosenAccountId) - .resource("stream/" + streamName) - .build()); - } - -} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java index 60195d4f..8307ed82 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -23,9 +23,7 @@ import lombok.NonNull; import lombok.experimental.Accessors; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; import software.amazon.awssdk.utils.Validate; -import software.amazon.kinesis.retrieval.KinesisClientFacade; import java.util.Optional; import java.util.regex.Matcher; @@ -42,9 +40,10 @@ public class StreamIdentifier { @NonNull private final String streamName; @Builder.Default - private Optional streamCreationEpochOptional = Optional.empty(); + private final Optional streamCreationEpochOptional = Optional.empty(); @Builder.Default - private final Optional streamARNOptional = Optional.empty(); + @EqualsAndHashCode.Exclude + private final Optional streamArnOptional = Optional.empty(); /** * Pattern for a serialized {@link StreamIdentifier}. The valid format is @@ -59,39 +58,25 @@ public class StreamIdentifier { * where {@code region} is the id representation of a {@link Region}. */ private static final Pattern STREAM_ARN_PATTERN = Pattern.compile( - "arn:aws:kinesis:(?[-a-z0-9]+):(?[0-9]{12}):stream/(?.+)"); + "arn:aws[^:]*:kinesis:(?[-a-z0-9]+):(?[0-9]{12}):stream/(?.+)"); /** * Serialize the current StreamIdentifier instance. * - * @return a String of {@code account:stream:creationEpoch[:region]} - * where {@code region} is the id representation of a {@link Region} - * and is optional. + * @return a String of {@code account:stream:creationEpoch} in multi-stream mode + * or {@link #streamName} in single-stream mode. */ public String serialize() { - if (!accountIdOptional.isPresent()) { + if (!streamCreationEpochOptional.isPresent()) { + // creation epoch is expected to be empty in single-stream mode return streamName; } - if (!streamCreationEpochOptional.isPresent()) { - // FIXME bias-for-action hack to simplify back-porting into KCL 1.x and facilitate the - // backwards-compatible requirement. There's a chicken-and-egg issue if DSS is - // called as the application is being configured (and before the client is rigged). - // Furthermore, if epoch isn't lazy-loaded here, the problem quickly spirals into - // systemic issues of concurrency and consistency (e.g., PeriodicShardSyncManager, - // Scheduler, DDB leases). We should look at leveraging dependency injection. - // (NOTE: not to inject the Kinesis client here, but to ensure the client is - // accessible elsewhere ASAP.) - final DescribeStreamSummaryResponse dss = KinesisClientFacade.describeStreamSummary( - streamARNOptional().get().toString()); - final long creationEpoch = dss.streamDescriptionSummary().streamCreationTimestamp().getEpochSecond(); - streamCreationEpochOptional = Optional.of(creationEpoch); - } - final char delimiter = ':'; - final StringBuilder sb = new StringBuilder(accountIdOptional.get()).append(delimiter) - .append(streamName).append(delimiter); - streamCreationEpochOptional.ifPresent(sb::append); + final StringBuilder sb = new StringBuilder() + .append(accountIdOptional.get()).append(delimiter) + .append(streamName).append(delimiter) + .append(streamCreationEpochOptional.get()); return sb.toString(); } @@ -101,129 +86,98 @@ public class StreamIdentifier { } /** - * Create a multi stream instance for StreamIdentifier from serialized stream identifier. + * Create a multi stream instance for StreamIdentifier from serialized stream identifier + * of format {@link #STREAM_IDENTIFIER_PATTERN} * - * @param serializationOrArn serialized {@link StreamIdentifier} or AWS ARN of a Kinesis stream - * - * @see #multiStreamInstance(String, Region) - * @see #serialize() + * @param streamIdentifierSer a String of {@code account:stream:creationEpoch} + * @return StreamIdentifier with {@link #accountIdOptional} and {@link #streamCreationEpochOptional} present */ - public static StreamIdentifier multiStreamInstance(String serializationOrArn) { - return multiStreamInstance(serializationOrArn, null); + public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) { + final Matcher matcher = STREAM_IDENTIFIER_PATTERN.matcher(streamIdentifierSer); + if (matcher.matches()) { + final String accountId = matcher.group("accountId"); + final String streamName = matcher.group("streamName"); + final Long creationEpoch = Long.valueOf(matcher.group("creationEpoch")); + + validateCreationEpoch(creationEpoch); + + return StreamIdentifier.builder() + .accountIdOptional(Optional.of(accountId)) + .streamName(streamName) + .streamCreationEpochOptional(Optional.of(creationEpoch)) + .build(); + } + + throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer); } /** - * Create a multi stream instance for StreamIdentifier from serialized stream identifier. + * Create a multi stream instance for StreamIdentifier from stream {@link Arn}. * - * @param serializationOrArn serialized {@link StreamIdentifier} or AWS ARN of a Kinesis stream - * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be - * processed are located. A null will default to the caller's region. - * - * @see #multiStreamInstance(String) - * @see #serialize() + * @param streamArn an {@link Arn} of format {@link #STREAM_ARN_PATTERN} + * @param creationEpoch Creation epoch of the stream. This value will + * reflect in the lease key and is assumed to be correct. (KCL could + * verify, but that creates issues for both bootstrapping and, with large + * KCL applications, API throttling against DescribeStreamSummary.) + * If this epoch is reused for two identically-named streams in the same + * account -- such as deleting and recreating a stream -- then KCL will + * be unable to differentiate leases between the old and new stream + * since the lease keys collide on this creation epoch. + * @return StreamIdentifier with {@link #accountIdOptional}, {@link #streamCreationEpochOptional}, + * and {@link #streamArnOptional} present */ - public static StreamIdentifier multiStreamInstance(String serializationOrArn, Region kinesisRegion) { - final StreamIdentifier fromSerialization = fromSerialization(serializationOrArn, kinesisRegion); - if (fromSerialization != null) { - return fromSerialization; - } - final StreamIdentifier fromArn = fromArn(serializationOrArn, kinesisRegion); - if (fromArn != null) { - return fromArn; - } + public static StreamIdentifier multiStreamInstance(Arn streamArn, long creationEpoch) { + validateArn(streamArn); + validateCreationEpoch(creationEpoch); - throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + serializationOrArn); + return StreamIdentifier.builder() + .accountIdOptional(streamArn.accountId()) + .streamName(streamArn.resource().resource()) + .streamCreationEpochOptional(Optional.of(creationEpoch)) + .streamArnOptional(Optional.of(streamArn)) + .build(); } /** * Create a single stream instance for StreamIdentifier from stream name. * - * @param streamNameOrArn stream name or AWS ARN of a Kinesis stream - * - * @see #singleStreamInstance(String, Region) + * @param streamName stream name of a Kinesis stream */ - public static StreamIdentifier singleStreamInstance(String streamNameOrArn) { - return singleStreamInstance(streamNameOrArn, null); - } - - /** - * Create a single stream instance for StreamIdentifier from the provided stream name and kinesisRegion. - * This method also constructs the optional StreamARN based on the region info. - * - * @param streamNameOrArn stream name or AWS ARN of a Kinesis stream - * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be - * processed are located. A null will default to the caller's region. - * - * @see #singleStreamInstance(String) - */ - public static StreamIdentifier singleStreamInstance(String streamNameOrArn, Region kinesisRegion) { - Validate.notEmpty(streamNameOrArn, "StreamName should not be empty"); - - final StreamIdentifier fromArn = fromArn(streamNameOrArn, kinesisRegion); - if (fromArn != null) { - return fromArn; - } + public static StreamIdentifier singleStreamInstance(String streamName) { + Validate.notEmpty(streamName, "StreamName should not be empty"); return StreamIdentifier.builder() - .streamName(streamNameOrArn) - .streamARNOptional(StreamARNUtil.getStreamARN(streamNameOrArn, kinesisRegion)) - .build(); - } - - /** - * Deserializes a StreamIdentifier from {@link #STREAM_IDENTIFIER_PATTERN}. - * - * @param input input string (e.g., ARN, serialized instance) to convert into an instance - * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be - * processed are located. A null will default to the caller's region. - * @return a StreamIdentifier instance if the pattern matched, otherwise null - */ - private static StreamIdentifier fromSerialization(final String input, final Region kinesisRegion) { - final Matcher matcher = STREAM_IDENTIFIER_PATTERN.matcher(input); - return matcher.matches() - ? toStreamIdentifier(matcher, matcher.group("creationEpoch"), kinesisRegion) : null; - } - - /** - * Constructs a StreamIdentifier from {@link #STREAM_ARN_PATTERN}. - * - * @param input input string (e.g., ARN, serialized instance) to convert into an instance - * @param kinesisRegion Kinesis client endpoint, and also where the stream(s) to be - * processed are located. A null will default to the caller's region. - * @return a StreamIdentifier instance if the pattern matched, otherwise null - */ - private static StreamIdentifier fromArn(final String input, final Region kinesisRegion) { - final Matcher matcher = STREAM_ARN_PATTERN.matcher(input); - if (matcher.matches()) { - final String arnRegion = matcher.group("region"); - final Region region = (arnRegion != null) ? Region.of(arnRegion) : kinesisRegion; - if ((kinesisRegion != null) && (region != kinesisRegion)) { - throw new IllegalArgumentException(String.format( - "Cannot create StreamIdentifier for a region other than %s: %s", kinesisRegion, input)); - } - return toStreamIdentifier(matcher, "", region); - } - return null; - } - - private static StreamIdentifier toStreamIdentifier(final Matcher matcher, final String matchedEpoch, - final Region kinesisRegion) { - final String accountId = matcher.group("accountId"); - final String streamName = matcher.group("streamName"); - final Optional creationEpoch = matchedEpoch.isEmpty() ? Optional.empty() - : Optional.of(Long.valueOf(matchedEpoch)); - final Optional arn = StreamARNUtil.getStreamARN(streamName, kinesisRegion, accountId); - - if (!creationEpoch.isPresent() && !arn.isPresent()) { - throw new IllegalArgumentException("Cannot create StreamIdentifier if missing both ARN and creation epoch"); - } - - return StreamIdentifier.builder() - .accountIdOptional(Optional.of(accountId)) .streamName(streamName) - .streamCreationEpochOptional(creationEpoch) - .streamARNOptional(arn) .build(); } + /** + * Create a single stream instance for StreamIdentifier from AWS Kinesis stream {@link Arn}. + * + * @param streamArn AWS ARN of a Kinesis stream + * @return StreamIdentifier with {@link #accountIdOptional} and {@link #streamArnOptional} present + */ + public static StreamIdentifier singleStreamInstance(Arn streamArn) { + validateArn(streamArn); + + return StreamIdentifier.builder() + .accountIdOptional(streamArn.accountId()) + .streamName(streamArn.resource().resource()) + .streamArnOptional(Optional.of(streamArn)) + .build(); + } + + private static void validateArn(Arn streamArn) { + if (!STREAM_ARN_PATTERN.matcher(streamArn.toString()).matches() || !streamArn.region().isPresent()) { + throw new IllegalArgumentException("Unable to create a StreamIdentifier from " + streamArn); + } + } + + private static void validateCreationEpoch(long creationEpoch) { + if (creationEpoch <= 0) { + throw new IllegalArgumentException( + "Creation epoch must be > 0; received " + creationEpoch); + } + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java deleted file mode 100644 index abe2822d..00000000 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SupplierCache.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import java.util.function.Supplier; - -import lombok.RequiredArgsConstructor; - -/** - * Caches results from a {@link Supplier}. Caching is especially useful when - * {@link Supplier#get()} is an expensive call that produces static results. - * - * @param result type - */ -@RequiredArgsConstructor -public class SupplierCache extends SynchronizedCache { - - private final Supplier supplier; - - /** - * Returns the cached result. If the cache is null, the supplier will be - * invoked to populate the cache. - * - * @return cached result which may be null - */ - public T get() { - return get(supplier); - } - -} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java deleted file mode 100644 index 3df241d3..00000000 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/SynchronizedCache.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import java.util.function.Supplier; - -/** - * A synchronized, "no frills" cache that preserves the first non-null value - * returned from a {@link Supplier}. - * - * @param result type - */ -public class SynchronizedCache { - - private volatile R result; - - /** - * Returns the cached result. If the cache is null, the supplier will be - * invoked to populate the cache. - * - * @param supplier supplier to invoke if the cache is null - * @return cached result which may be null - */ - protected R get(final Supplier supplier) { - if (result == null) { - synchronized (this) { - // double-check lock - if (result == null) { - result = supplier.get(); - } - } - } - return result; - } - -} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java index 04b7c795..f22c631a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -223,8 +223,7 @@ public class KinesisShardDetector implements ShardDetector { ListShardsRequest.Builder builder = KinesisRequestsBuilder.listShardsRequestBuilder(); if (StringUtils.isEmpty(nextToken)) { builder.streamName(streamIdentifier.streamName()).shardFilter(shardFilter); - streamIdentifier.streamARNOptional().ifPresent(arn -> builder.streamARN(arn.toString())); - + streamIdentifier.streamArnOptional().ifPresent(arn -> builder.streamARN(arn.toString())); } else { builder.nextToken(nextToken); } @@ -313,7 +312,7 @@ public class KinesisShardDetector implements ShardDetector { .streamName(streamIdentifier.streamName()) .shardIteratorType(ShardIteratorType.LATEST) .shardId(shardId); - streamIdentifier.streamARNOptional().ifPresent(arn -> requestBuilder.streamARN(arn.toString())); + streamIdentifier.streamArnOptional().ifPresent(arn -> requestBuilder.streamARN(arn.toString())); final GetShardIteratorRequest getShardIteratorRequest = requestBuilder.build(); final GetShardIteratorResponse getShardIteratorResponse = diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java index ee2850ed..9b5f85c3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/SingleStreamTracker.java @@ -21,7 +21,7 @@ import java.util.List; import lombok.EqualsAndHashCode; import lombok.NonNull; import lombok.ToString; -import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.arns.Arn; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; @@ -49,8 +49,8 @@ public class SingleStreamTracker implements StreamTracker { this(StreamIdentifier.singleStreamInstance(streamName)); } - public SingleStreamTracker(String streamName, Region region) { - this(StreamIdentifier.singleStreamInstance(streamName, region)); + public SingleStreamTracker(Arn streamArn) { + this(StreamIdentifier.singleStreamInstance(streamArn)); } public SingleStreamTracker(StreamIdentifier streamIdentifier) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java deleted file mode 100644 index d2f90d3a..00000000 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisClientFacade.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.retrieval; - -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; - -import lombok.NonNull; -import lombok.extern.slf4j.Slf4j; -import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest; -import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; -import software.amazon.awssdk.services.kinesis.model.KinesisException; -import software.amazon.awssdk.services.kinesis.model.LimitExceededException; -import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; -import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; -import software.amazon.kinesis.common.KinesisRequestsBuilder; - -/** - * Facade pattern to simplify interactions with a {@link KinesisAsyncClient}. - */ -@Slf4j -public final class KinesisClientFacade { - - /** - * Reusable {@link AWSExceptionManager}. - *

- * N.B. This instance is mutable, but thread-safe for read-only use. - *

- */ - private static final AWSExceptionManager AWS_EXCEPTION_MANAGER; - - // FIXME dependency injection - private static KinesisAsyncClient kinesisClient; - - static { - AWS_EXCEPTION_MANAGER = new AWSExceptionManager(); - AWS_EXCEPTION_MANAGER.add(KinesisException.class, t -> t); - AWS_EXCEPTION_MANAGER.add(LimitExceededException.class, t -> t); - AWS_EXCEPTION_MANAGER.add(ResourceInUseException.class, t -> t); - AWS_EXCEPTION_MANAGER.add(ResourceNotFoundException.class, t -> t); - } - - static void initialize(final KinesisAsyncClient client) { - kinesisClient = client; - } - - public static DescribeStreamSummaryResponse describeStreamSummary(final String streamArn) { - final DescribeStreamSummaryRequest request = KinesisRequestsBuilder - .describeStreamSummaryRequestBuilder().streamARN(streamArn).build(); - final ServiceCallerSupplier dss = - () -> kinesisClient.describeStreamSummary(request).get(); - return retryWhenThrottled(dss, 3, streamArn, "DescribeStreamSummary"); - } - - // FIXME code lifted-and-shifted from FanOutConsumerRegistration; that class - // (and others) should not be responsible for interacting directly with - // the thread-safe Kinesis client (and handling retries, etc.) - private static T retryWhenThrottled( - @NonNull final ServiceCallerSupplier retriever, - final int maxRetries, - final String streamArn, - @NonNull final String apiName) { - LimitExceededException finalException = null; - - int retries = maxRetries; - while (retries > 0) { - try { - try { - return retriever.get(); - } catch (ExecutionException e) { - throw AWS_EXCEPTION_MANAGER.apply(e.getCause()); - } catch (InterruptedException e) { - throw KinesisException.create("Unable to complete " + apiName, e); - } catch (TimeoutException te) { - log.info("Timed out waiting for " + apiName + " for " + streamArn); - } - } catch (LimitExceededException e) { - log.info("{} : Throttled while calling {} API, will backoff.", streamArn, apiName); - try { - Thread.sleep(1000 + (long) (Math.random() * 100)); - } catch (InterruptedException ie) { - log.debug("Sleep interrupted, shutdown invoked."); - } - finalException = e; - } - retries--; - } - - if (finalException == null) { - throw new IllegalStateException(streamArn + " : Exhausted retries while calling " + apiName); - } - - throw finalException; - } - - @FunctionalInterface - private interface ServiceCallerSupplier { - T get() throws ExecutionException, InterruptedException, TimeoutException; - } - -} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 153faf70..082dd565 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -22,6 +22,7 @@ import lombok.NonNull; import lombok.Setter; import lombok.ToString; import lombok.experimental.Accessors; +import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.common.DeprecationUtils; @@ -119,9 +120,12 @@ public class RetrievalConfig { public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName, @NonNull String applicationName) { - this(kinesisAsyncClient, - new SingleStreamTracker(streamName, kinesisAsyncClient.serviceClientConfiguration().region()), - applicationName); + this(kinesisAsyncClient, new SingleStreamTracker(streamName), applicationName); + } + + public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull Arn streamArn, + @NonNull String applicationName) { + this(kinesisAsyncClient, new SingleStreamTracker(streamArn), applicationName); } public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull StreamTracker streamTracker, @@ -131,8 +135,6 @@ public class RetrievalConfig { this.applicationName = applicationName; this.appStreamTracker = DeprecationUtils.convert(streamTracker, singleStreamTracker -> singleStreamTracker.streamConfigList().get(0)); - - KinesisClientFacade.initialize(kinesisAsyncClient); } /** diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java index 96f8f851..65da2b32 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java @@ -238,7 +238,7 @@ public class KinesisDataFetcher implements DataFetcher { GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder() .streamName(streamIdentifier.streamName()).shardId(shardId); - streamIdentifier.streamARNOptional().ifPresent(arn -> builder.streamARN(arn.toString())); + streamIdentifier.streamArnOptional().ifPresent(arn -> builder.streamARN(arn.toString())); GetShardIteratorRequest request; if (isIteratorRestart) { @@ -321,7 +321,7 @@ public class KinesisDataFetcher implements DataFetcher { public GetRecordsRequest getGetRecordsRequest(String nextIterator) { GetRecordsRequest.Builder builder = KinesisRequestsBuilder.getRecordsRequestBuilder() .shardIterator(nextIterator).limit(maxRecords); - streamIdentifier.streamARNOptional().ifPresent(arn -> builder.streamARN(arn.toString())); + streamIdentifier.streamArnOptional().ifPresent(arn -> builder.streamARN(arn.toString())); return builder.build(); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java index e1de0981..d84b90f7 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/ConfigsBuilderTest.java @@ -26,6 +26,8 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -33,12 +35,12 @@ import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; -import software.amazon.kinesis.utils.MockObjectHelper; @RunWith(MockitoJUnitRunner.class) public class ConfigsBuilderTest { - private final KinesisAsyncClient mockKinesisClient = MockObjectHelper.createKinesisClient(); + @Mock + private KinesisAsyncClient mockKinesisClient; @Mock private DynamoDbAsyncClient mockDynamoClient; @@ -53,18 +55,24 @@ public class ConfigsBuilderTest { private static final String WORKER_IDENTIFIER = "worker-id"; @Test - public void testTrackerConstruction() { + public void testSingleStreamTrackerConstruction() { final String streamName = "single-stream"; - final ConfigsBuilder configByName = createConfig(streamName); - final ConfigsBuilder configBySingleTracker = createConfig(new SingleStreamTracker(streamName)); + final Arn streamArn = createArn(streamName); - for (final ConfigsBuilder cb : Arrays.asList(configByName, configBySingleTracker)) { + for (final ConfigsBuilder cb : Arrays.asList( + createConfig(streamName), + createConfig(new SingleStreamTracker(streamName)), + createConfig(streamArn), + createConfig(new SingleStreamTracker(streamArn)))) { assertEquals(Optional.empty(), cb.appStreamTracker().left()); assertEquals(streamName, cb.appStreamTracker().right().get()); assertEquals(streamName, cb.streamTracker().streamConfigList().get(0).streamIdentifier().streamName()); assertFalse(cb.streamTracker().isMultiStream()); } + } + @Test + public void testMultiStreamTrackerConstruction() { final StreamTracker mockMultiStreamTracker = mock(MultiStreamTracker.class); final ConfigsBuilder configByMultiTracker = createConfig(mockMultiStreamTracker); assertEquals(Optional.empty(), configByMultiTracker.appStreamTracker().right()); @@ -78,9 +86,25 @@ public class ConfigsBuilderTest { mockCloudWatchClient, WORKER_IDENTIFIER, mockShardProcessorFactory); } + private ConfigsBuilder createConfig(Arn streamArn) { + // intentional invocation of constructor where streamArn is an Arn + return new ConfigsBuilder(streamArn, APPLICATION_NAME, mockKinesisClient, mockDynamoClient, + mockCloudWatchClient, WORKER_IDENTIFIER, mockShardProcessorFactory); + } + private ConfigsBuilder createConfig(StreamTracker streamTracker) { return new ConfigsBuilder(streamTracker, APPLICATION_NAME, mockKinesisClient, mockDynamoClient, mockCloudWatchClient, WORKER_IDENTIFIER, mockShardProcessorFactory); } -} \ No newline at end of file + private static Arn createArn(String streamName) { + return Arn.builder() + .partition("aws") + .service("kinesis") + .region(Region.US_EAST_1.id()) + .accountId("123456789012") + .resource("stream/" + streamName) + .build(); + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java deleted file mode 100644 index 2f55af4b..00000000 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/FunctionCacheTest.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import java.util.function.Function; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class FunctionCacheTest { - - @Mock - private Function mockFunction; - - private FunctionCache cache; - - @Before - public void setUp() { - cache = new FunctionCache<>(mockFunction); - } - - /** - * Test that the cache stops invoking the encapsulated {@link Function} - * after it returns a non-null value. - */ - @Test - public void testCache() { - final int expectedValue = 3; - when(mockFunction.apply(expectedValue)).thenReturn(expectedValue); - - assertNull(cache.get(1)); - assertNull(cache.get(2)); - assertEquals(expectedValue, cache.get(3)); - assertEquals(expectedValue, cache.get(4)); - assertEquals(expectedValue, cache.get(5)); - verify(mockFunction, times(expectedValue)).apply(anyInt()); - } -} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java deleted file mode 100644 index b6009b43..00000000 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamARNUtilTest.java +++ /dev/null @@ -1,201 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; -import software.amazon.awssdk.arns.Arn; -import software.amazon.awssdk.awscore.exception.AwsServiceException; -import software.amazon.awssdk.core.exception.SdkClientException; -import software.amazon.awssdk.http.SdkHttpClient; -import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.sts.StsClient; -import software.amazon.awssdk.services.sts.StsClientBuilder; -import software.amazon.awssdk.services.sts.model.GetCallerIdentityResponse; - -import java.lang.reflect.Field; -import java.util.Optional; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyZeroInteractions; -import static org.mockito.Mockito.when; - -@RunWith(PowerMockRunner.class) -@PrepareForTest({ StreamARNUtil.class, StsClient.class, UrlConnectionHttpClient.class }) -public class StreamARNUtilTest { - private static final String STS_RESPONSE_ARN_FORMAT = "arn:aws:sts::%s:assumed-role/Admin/alias"; - private static final String KINESIS_STREAM_ARN_FORMAT = "arn:aws:kinesis:us-east-1:%s:stream/%s"; - - /** - * Original {@link SupplierCache} that is constructed on class load. - */ - private static final SupplierCache ORIGINAL_CACHE = Whitebox.getInternalState( - StreamARNUtil.class, "CALLER_IDENTITY_ARN"); - - private static final String ACCOUNT_ID = "12345"; - - private static final String STREAM_NAME = StreamARNUtilTest.class.getSimpleName(); - - @Mock - private StsClientBuilder mockStsClientBuilder; - - @Mock - private StsClient mockStsClient; - - private SupplierCache spySupplierCache; - - @Before - public void setUp() throws Exception { - MockitoAnnotations.initMocks(this); - - spySupplierCache = spy(ORIGINAL_CACHE); - setUpSupplierCache(spySupplierCache); - - final Arn defaultArn = toArn(STS_RESPONSE_ARN_FORMAT, ACCOUNT_ID); - doReturn(defaultArn).when(spySupplierCache).get(); - } - - private void setUpSts() { - PowerMockito.mockStatic(StsClient.class); - PowerMockito.mockStatic(UrlConnectionHttpClient.class); - - when(UrlConnectionHttpClient.builder()).thenReturn(mock(UrlConnectionHttpClient.Builder.class)); - when(StsClient.builder()).thenReturn(mockStsClientBuilder); - when(mockStsClientBuilder.httpClient(any(SdkHttpClient.class))).thenReturn(mockStsClientBuilder); - when(mockStsClientBuilder.build()).thenReturn(mockStsClient); - - // bypass the spy so the Sts clients are called - when(spySupplierCache.get()).thenCallRealMethod(); - } - - /** - * Wrap and embed the original {@link SupplierCache} with a spy to avoid - * one-and-done cache behavior, provide each test precise control over - * return values, and enable the ability to verify interactions via Mockito. - */ - static void setUpSupplierCache(final SupplierCache cache) throws Exception { - final Field f = StreamARNUtil.class.getDeclaredField("CALLER_IDENTITY_ARN"); - f.setAccessible(true); - f.set(null, cache); - f.setAccessible(false); - } - - @Test - public void testGetStreamARNHappyCase() { - getStreamArn(); - - verify(spySupplierCache).get(); - } - - @Test - public void testGetStreamARNFromCache() { - final Optional actualStreamARNOptional1 = getStreamArn(); - final Optional actualStreamARNOptional2 = getStreamArn(); - - verify(spySupplierCache, times(2)).get(); - assertEquals(actualStreamARNOptional1, actualStreamARNOptional2); - } - - @Test - public void testGetStreamARNReturnsEmptyOnSTSError() { - setUpSts(); - - // Optional.empty() is expected when there is an error with the STS call and STS returns empty Arn - when(mockStsClient.getCallerIdentity()) - .thenThrow(AwsServiceException.builder().message("testAwsServiceException").build()) - .thenThrow(SdkClientException.builder().message("testSdkClientException").build()); - - assertEquals(Optional.empty(), StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1)); - assertEquals(Optional.empty(), StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1)); - verify(mockStsClient, times(2)).getCallerIdentity(); - verify(spySupplierCache, times(2)).get(); - } - - @Test(expected = IllegalStateException.class) - public void testStsResponseWithoutAccountId() { - setUpSts(); - - final Arn arnWithoutAccountId = toArn(STS_RESPONSE_ARN_FORMAT, ""); - assertEquals(Optional.empty(), arnWithoutAccountId.accountId()); - - final GetCallerIdentityResponse identityResponse = GetCallerIdentityResponse.builder() - .arn(arnWithoutAccountId.toString()).build(); - when(mockStsClient.getCallerIdentity()).thenReturn(identityResponse); - - try { - StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); - } finally { - verify(mockStsClient).getCallerIdentity(); - } - } - - @Test - public void testGetStreamARNReturnsEmptyOnInvalidKinesisRegion() { - // Optional.empty() is expected when kinesis region is not set correctly - Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(STREAM_NAME, null); - assertEquals(Optional.empty(), actualStreamARNOptional); - verifyZeroInteractions(mockStsClient); - verifyZeroInteractions(spySupplierCache); - } - - @Test - public void testGetStreamARNWithProvidedAccountIDAndIgnoredSTSResult() { - // If the account id is provided in the StreamIdentifier, it will override the result (account id) returned by sts - final String cachedAccountId = "111111111111"; - final String providedAccountId = "222222222222"; - - final Arn cachedArn = toArn(STS_RESPONSE_ARN_FORMAT, cachedAccountId); - when(spySupplierCache.get()).thenReturn(cachedArn); - - final Optional actualStreamARNOptional = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1, - providedAccountId); - final Arn expectedStreamARN = toArn(KINESIS_STREAM_ARN_FORMAT, providedAccountId, STREAM_NAME); - - verify(spySupplierCache).get(); - verifyZeroInteractions(mockStsClient); - assertTrue(actualStreamARNOptional.isPresent()); - assertEquals(expectedStreamARN, actualStreamARNOptional.get()); - } - - private static Optional getStreamArn() { - final Optional actualArn = StreamARNUtil.getStreamARN(STREAM_NAME, Region.US_EAST_1); - final Arn expectedArn = toArn(KINESIS_STREAM_ARN_FORMAT, ACCOUNT_ID, STREAM_NAME); - - assertTrue(actualArn.isPresent()); - assertEquals(expectedArn, actualArn.get()); - - return actualArn; - } - - private static Arn toArn(final String format, final Object... params) { - return Arn.fromString(String.format(format, params)); - } - -} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java index b3f4991b..d511864f 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/StreamIdentifierTest.java @@ -1,11 +1,7 @@ package software.amazon.kinesis.common; import org.junit.Assert; -import org.junit.BeforeClass; import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.regions.Region; @@ -13,27 +9,16 @@ import java.util.Arrays; import java.util.Optional; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotEquals; -import static org.mockito.Mockito.when; -import static org.powermock.api.mockito.PowerMockito.mockStatic; -import static org.powermock.api.mockito.PowerMockito.verifyStatic; -import static software.amazon.kinesis.common.StreamARNUtil.getStreamARN; -@RunWith(PowerMockRunner.class) -@PrepareForTest(StreamARNUtil.class) public class StreamIdentifierTest { private static final String STREAM_NAME = "stream-name"; + private static final String PARTITION = "aws"; + private static final String SERVICE = "kinesis"; private static final Region KINESIS_REGION = Region.US_WEST_1; private static final String TEST_ACCOUNT_ID = "123456789012"; + private static final String RESOURCE = "stream/" + STREAM_NAME; private static final long EPOCH = 1680616058L; - - private static final Arn DEFAULT_ARN = toArn(KINESIS_REGION); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - StreamARNUtilTest.setUpSupplierCache(new SupplierCache<>(() -> DEFAULT_ARN)); - } + private static final Arn DEFAULT_ARN = createArn(); /** * Test patterns that should match a serialization regex. @@ -51,74 +36,86 @@ public class StreamIdentifierTest { @Test public void testMultiStreamDeserializationFail() { for (final String pattern : Arrays.asList( - // arn examples - "arn:aws:kinesis::123456789012:stream/stream-name", // missing region - "arn:aws:kinesis:region::stream/stream-name", // missing account id - "arn:aws:kinesis:region:123456789:stream/stream-name", // account id not 12 digits - "arn:aws:kinesis:region:123456789abc:stream/stream-name", // 12char alphanumeric account id - "arn:aws:kinesis:region:123456789012:stream/", // missing stream-name - // serialization examples ":stream-name:123", // missing account id // "123456789:stream-name:123", // account id not 12 digits "123456789abc:stream-name:123", // 12char alphanumeric account id "123456789012::123", // missing stream name "123456789012:stream-name", // missing delimiter and creation epoch "123456789012:stream-name:", // missing creation epoch + "123456789012:stream-name:-123", // negative creation epoch "123456789012:stream-name:abc", // non-numeric creation epoch "" )) { try { StreamIdentifier.multiStreamInstance(pattern); - Assert.fail(pattern + " should not have created a StreamIdentifier"); + Assert.fail("Serialization " + pattern + " should not have created a StreamIdentifier"); } catch (final IllegalArgumentException iae) { // expected; ignore } } } + /** + * Test ARNs that should not match a valid AWS Kinesis stream ARN. + */ @Test - public void testInstanceFromArn() { - final Arn arn = toArn(KINESIS_REGION); - final StreamIdentifier single = StreamIdentifier.singleStreamInstance(arn.toString()); - final StreamIdentifier multi = StreamIdentifier.multiStreamInstance(arn.toString()); - - assertEquals(single, multi); - assertEquals(Optional.empty(), single.streamCreationEpochOptional()); - assertActualStreamIdentifierExpected(arn, single); + public void testMultiStreamByArnWithInvalidStreamArnFail() { + for (final Arn invalidStreamArn : Arrays.asList( + createArn("abc", SERVICE, KINESIS_REGION, TEST_ACCOUNT_ID, RESOURCE), // invalid partition + createArn(PARTITION, "dynamodb", KINESIS_REGION, TEST_ACCOUNT_ID, RESOURCE), // incorrect service + createArn(PARTITION, SERVICE, null, TEST_ACCOUNT_ID, RESOURCE), // missing region + createArn(PARTITION, SERVICE, KINESIS_REGION, null, RESOURCE), // missing account id + createArn(PARTITION, SERVICE, KINESIS_REGION, "123456789", RESOURCE), // account id not 12 digits + createArn(PARTITION, SERVICE, KINESIS_REGION, "123456789abc", RESOURCE), // 12char alphanumeric account id + createArn(PARTITION, SERVICE, KINESIS_REGION, TEST_ACCOUNT_ID, "table/name"), // incorrect resource type + Arn.fromString("arn:aws:dynamodb:us-east-2:123456789012:table/myDynamoDBTable") // valid ARN for incorrect resource + )) { + try { + StreamIdentifier.multiStreamInstance(invalidStreamArn, EPOCH); + Assert.fail("Arn " + invalidStreamArn + " should not have created a StreamIdentifier"); + } catch (final IllegalArgumentException iae) { + // expected; ignore + } + } } @Test(expected = IllegalArgumentException.class) - public void testInstanceWithoutEpochOrArn() { - mockStatic(StreamARNUtil.class); - when(getStreamARN(STREAM_NAME, KINESIS_REGION, TEST_ACCOUNT_ID)) - .thenReturn(Optional.empty()); + public void testNegativeCreationEpoch() { + StreamIdentifier.multiStreamInstance(DEFAULT_ARN, -123); + } - try { - StreamIdentifier.singleStreamInstance(DEFAULT_ARN.toString()); - } finally { - verifyStatic(StreamARNUtil.class); - getStreamARN(STREAM_NAME, KINESIS_REGION, TEST_ACCOUNT_ID); - } + @Test(expected = IllegalArgumentException.class) + public void testZeroCreationEpoch() { + StreamIdentifier.multiStreamInstance(DEFAULT_ARN, 0); + } + + @Test + public void testSingleStreamInstanceFromArn() { + final StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(DEFAULT_ARN); + + assertActualStreamIdentifierExpected(DEFAULT_ARN, actualStreamIdentifier); + assertEquals(Optional.empty(), actualStreamIdentifier.streamCreationEpochOptional()); + assertEquals(actualStreamIdentifier.streamName(), actualStreamIdentifier.serialize()); + } + + @Test + public void testMultiStreamInstanceFromArn() { + final StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance(DEFAULT_ARN, EPOCH); + + assertActualStreamIdentifierExpected(DEFAULT_ARN, actualStreamIdentifier); + assertEquals(Optional.of(EPOCH), actualStreamIdentifier.streamCreationEpochOptional()); + assertEquals(serialize(), actualStreamIdentifier.serialize()); } @Test public void testSingleStreamInstanceWithName() { StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME); - assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); - assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); - assertFalse(actualStreamIdentifier.streamARNOptional().isPresent()); + assertEquals(Optional.empty(), actualStreamIdentifier.streamCreationEpochOptional()); + assertEquals(Optional.empty(), actualStreamIdentifier.accountIdOptional()); + assertEquals(Optional.empty(), actualStreamIdentifier.streamArnOptional()); assertEquals(STREAM_NAME, actualStreamIdentifier.streamName()); } - @Test - public void testSingleStreamInstanceWithNameAndRegion() { - StreamIdentifier actualStreamIdentifier = StreamIdentifier.singleStreamInstance(STREAM_NAME, KINESIS_REGION); - assertFalse(actualStreamIdentifier.streamCreationEpochOptional().isPresent()); - assertFalse(actualStreamIdentifier.accountIdOptional().isPresent()); - assertEquals(STREAM_NAME, actualStreamIdentifier.streamName()); - assertEquals(Optional.of(DEFAULT_ARN), actualStreamIdentifier.streamARNOptional()); - } - @Test public void testMultiStreamInstanceWithIdentifierSerialization() { StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance(serialize()); @@ -126,34 +123,10 @@ public class StreamIdentifierTest { assertEquals(Optional.of(EPOCH), actualStreamIdentifier.streamCreationEpochOptional()); } - /** - * When KCL's Kinesis endpoint is a region, it lacks visibility to streams - * in other regions. Therefore, when the endpoint and ARN conflict, an - * Exception should be thrown. - */ - @Test(expected = IllegalArgumentException.class) - public void testConflictOnRegions() { - final Region arnRegion = Region.US_GOV_EAST_1; - assertNotEquals(arnRegion, KINESIS_REGION); - - StreamIdentifier.multiStreamInstance(toArn(arnRegion).toString(), KINESIS_REGION); - } - - @Test - public void testMultiStreamInstanceWithoutRegionSerialized() { - StreamIdentifier actualStreamIdentifier = StreamIdentifier.multiStreamInstance( - serialize(), KINESIS_REGION); - assertActualStreamIdentifierExpected(actualStreamIdentifier); - } - - private void assertActualStreamIdentifierExpected(StreamIdentifier actual) { - assertActualStreamIdentifierExpected(DEFAULT_ARN, actual); - } - private void assertActualStreamIdentifierExpected(Arn expectedArn, StreamIdentifier actual) { assertEquals(STREAM_NAME, actual.streamName()); assertEquals(Optional.of(TEST_ACCOUNT_ID), actual.accountIdOptional()); - assertEquals(Optional.ofNullable(expectedArn), actual.streamARNOptional()); + assertEquals(Optional.ofNullable(expectedArn), actual.streamArnOptional()); } /** @@ -163,11 +136,18 @@ public class StreamIdentifierTest { return String.join(":", TEST_ACCOUNT_ID, STREAM_NAME, Long.toString(EPOCH)); } - private static Arn toArn(final Region region) { - return Arn.builder().partition("aws").service("kinesis") - .accountId(TEST_ACCOUNT_ID) - .resource("stream/" + STREAM_NAME) - .region(region.toString()) + private static Arn createArn() { + return createArn(PARTITION, SERVICE, KINESIS_REGION, TEST_ACCOUNT_ID, RESOURCE); + } + + private static Arn createArn(String partition, String service, Region region, String account, String resource) { + return Arn.builder() + .partition(partition) + .service(service) + .region(region != null ? region.id() : null) + .accountId(account) + .resource(resource) .build(); } + } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java deleted file mode 100644 index a0bde098..00000000 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SupplierCacheTest.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import java.util.function.Supplier; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class SupplierCacheTest { - - private static final Object DUMMY_RESULT = SupplierCacheTest.class; - - @Mock - private Supplier mockSupplier; - - private SupplierCache cache; - - @Before - public void setUp() { - cache = new SupplierCache<>(mockSupplier); - } - - @Test - public void testCache() { - when(mockSupplier.get()).thenReturn(DUMMY_RESULT); - - final Object result1 = cache.get(); - final Object result2 = cache.get(); - - assertEquals(DUMMY_RESULT, result1); - assertSame(result1, result2); - verify(mockSupplier).get(); - } - - @Test - public void testCacheWithNullResult() { - when(mockSupplier.get()).thenReturn(null).thenReturn(DUMMY_RESULT); - - final Object result1 = cache.get(); - final Object result2 = cache.get(); - - assertNull(result1); - assertEquals(DUMMY_RESULT, result2); - verify(mockSupplier, times(2)).get(); - } -} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java deleted file mode 100644 index bad3f3cf..00000000 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/common/SynchronizedCacheTest.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.common; - -import java.util.function.Supplier; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertSame; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class SynchronizedCacheTest { - - private static final Object DUMMY_RESULT = SynchronizedCacheTest.class; - - @Mock - private Supplier mockSupplier; - - private final SynchronizedCache cache = new SynchronizedCache<>(); - - @Test - public void testCache() { - when(mockSupplier.get()).thenReturn(DUMMY_RESULT); - - final Object result1 = cache.get(mockSupplier); - final Object result2 = cache.get(mockSupplier); - - assertEquals(DUMMY_RESULT, result1); - assertSame(result1, result2); - verify(mockSupplier).get(); - } - - @Test - public void testCacheWithNullResult() { - when(mockSupplier.get()).thenReturn(null).thenReturn(DUMMY_RESULT); - - assertNull(cache.get(mockSupplier)); - assertEquals(DUMMY_RESULT, cache.get(mockSupplier)); - assertEquals(DUMMY_RESULT, cache.get(mockSupplier)); - verify(mockSupplier, times(2)).get(); - } -} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 1be0a9d4..46918f62 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -112,7 +112,6 @@ import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; import software.amazon.kinesis.retrieval.RetrievalFactory; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import software.amazon.kinesis.utils.MockObjectHelper; /** * @@ -138,6 +137,7 @@ public class SchedulerTest { private ProcessorConfig processorConfig; private RetrievalConfig retrievalConfig; + @Mock private KinesisAsyncClient kinesisClient; @Mock private DynamoDbAsyncClient dynamoDBClient; @@ -180,7 +180,6 @@ public class SchedulerTest { lifecycleConfig = new LifecycleConfig(); metricsConfig = new MetricsConfig(cloudWatchClient, namespace); processorConfig = new ProcessorConfig(shardRecordProcessorFactory); - kinesisClient = MockObjectHelper.createKinesisClient(); retrievalConfig = new RetrievalConfig(kinesisClient, streamName, applicationName) .retrievalFactory(retrievalFactory); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java deleted file mode 100644 index c2095d87..00000000 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/KinesisClientFacadeTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright 2023 Amazon.com, Inc. or its affiliates. - * Licensed under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package software.amazon.kinesis.retrieval; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - -import static org.junit.Assert.assertEquals; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -import static software.amazon.kinesis.retrieval.KinesisClientFacade.describeStreamSummary; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; -import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest; -import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; -import software.amazon.awssdk.services.kinesis.model.LimitExceededException; - -@RunWith(MockitoJUnitRunner.class) -public class KinesisClientFacadeTest { - - @Mock - private KinesisAsyncClient mockKinesisClient; - - @Before - public void setUp() { - KinesisClientFacade.initialize(mockKinesisClient); - } - - @Test - public void testDescribeStreamSummary() { - final DescribeStreamSummaryResponse expectedResponse = DescribeStreamSummaryResponse.builder().build(); - when(mockKinesisClient.describeStreamSummary(any(DescribeStreamSummaryRequest.class))) - .thenReturn(CompletableFuture.completedFuture(expectedResponse)); - - final DescribeStreamSummaryResponse actualResponse = describeStreamSummary("narf"); - assertEquals(expectedResponse, actualResponse); - - verify(mockKinesisClient).describeStreamSummary(any(DescribeStreamSummaryRequest.class)); - } - - @Test - public void testDescribeStreamSummaryRetries() throws Exception { - final DescribeStreamSummaryResponse expectedResponse = DescribeStreamSummaryResponse.builder().build(); - final CompletableFuture mockFuture = mock(CompletableFuture.class); - final ExecutionException executionException = new ExecutionException(LimitExceededException.builder().build()); - - when(mockKinesisClient.describeStreamSummary(any(DescribeStreamSummaryRequest.class))) - .thenReturn(mockFuture); - when(mockFuture.get()) - .thenThrow(executionException) - .thenThrow(executionException) - .thenReturn(expectedResponse); - - final DescribeStreamSummaryResponse actualResponse = describeStreamSummary("retry me plz"); - assertEquals(expectedResponse, actualResponse); - - verify(mockKinesisClient, times(3)).describeStreamSummary(any(DescribeStreamSummaryRequest.class)); - verify(mockFuture, times(3)).get(); - } -} \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java index 49cf4673..0f8273b8 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/RetrievalConfigTest.java @@ -18,19 +18,21 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.SingleStreamTracker; import software.amazon.kinesis.processor.StreamTracker; -import software.amazon.kinesis.utils.MockObjectHelper; @RunWith(MockitoJUnitRunner.class) public class RetrievalConfigTest { private static final String APPLICATION_NAME = RetrievalConfigTest.class.getSimpleName(); + @Mock private KinesisAsyncClient mockKinesisClient; @Mock @@ -38,24 +40,28 @@ public class RetrievalConfigTest { @Before public void setUp() { - mockKinesisClient = MockObjectHelper.createKinesisClient(true); when(mockMultiStreamTracker.isMultiStream()).thenReturn(true); } @Test - public void testTrackerConstruction() { + public void testSingleStreamTrackerConstruction() { final String streamName = "single-stream"; - final RetrievalConfig configByName = createConfig(streamName); - final SingleStreamTracker singleTracker = new SingleStreamTracker(streamName); - final RetrievalConfig configBySingleTracker = createConfig(singleTracker); + final Arn streamArn = createArn(streamName); - for (final RetrievalConfig rc : Arrays.asList(configByName, configBySingleTracker)) { + for (final RetrievalConfig rc : Arrays.asList( + createConfig(streamName), + createConfig(new SingleStreamTracker(streamName)), + createConfig(streamArn), + createConfig(new SingleStreamTracker(streamArn)))) { assertEquals(Optional.empty(), rc.appStreamTracker().left()); - assertEquals(singleTracker, rc.streamTracker()); + assertEquals(streamName, rc.streamTracker().streamConfigList().get(0).streamIdentifier().streamName()); assertEquals(1, rc.streamTracker().streamConfigList().size()); assertFalse(rc.streamTracker().isMultiStream()); } + } + @Test + public void testMultiStreamTrackerConstruction() { final StreamTracker mockMultiStreamTracker = mock(MultiStreamTracker.class); final RetrievalConfig configByMultiTracker = createConfig(mockMultiStreamTracker); assertEquals(Optional.empty(), configByMultiTracker.appStreamTracker().right()); @@ -110,8 +116,22 @@ public class RetrievalConfigTest { return new RetrievalConfig(mockKinesisClient, streamName, APPLICATION_NAME); } + private RetrievalConfig createConfig(Arn streamArn) { + return new RetrievalConfig(mockKinesisClient, streamArn, APPLICATION_NAME); + } + private RetrievalConfig createConfig(StreamTracker streamTracker) { return new RetrievalConfig(mockKinesisClient, streamTracker, APPLICATION_NAME); } + private static Arn createArn(String streamName) { + return Arn.builder() + .partition("aws") + .service("kinesis") + .region(Region.US_EAST_1.id()) + .accountId("123456789012") + .resource("stream/" + streamName) + .build(); + } + } \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java deleted file mode 100644 index 4fd050be..00000000 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/MockObjectHelper.java +++ /dev/null @@ -1,31 +0,0 @@ -package software.amazon.kinesis.utils; - -import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.awssdk.services.kinesis.KinesisServiceClientConfiguration; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public final class MockObjectHelper { - - public static KinesisAsyncClient createKinesisClient() { - return createKinesisClient(Region.US_EAST_1); - } - - /** - * @param isRegionDummy a boolean to determine whether to use a null value for the Kinesis client's region. - * @return - */ - public static KinesisAsyncClient createKinesisClient(boolean isRegionDummy) { - return isRegionDummy ? createKinesisClient(null) : createKinesisClient(); - } - - public static KinesisAsyncClient createKinesisClient(Region region) { - KinesisAsyncClient kinesisClient = mock(KinesisAsyncClient.class); - when(kinesisClient.serviceClientConfiguration()). - thenReturn(KinesisServiceClientConfiguration.builder().region(region).build()); - return kinesisClient; - } - -} From 8b3f957db4a0e02d5813aea4250e13cfcc8f1846 Mon Sep 17 00:00:00 2001 From: lucienlu-aws <132623944+lucienlu-aws@users.noreply.github.com> Date: Fri, 19 May 2023 16:25:45 -0700 Subject: [PATCH 64/88] Preparation for v2.5.0 (#1113) * Preparation for v2.5.0 --- CHANGELOG.md | 13 +++++++++++++ amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 5 files changed, 17 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 58f48a9f..1c494773 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,19 @@ For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md) --- +### Release 2.5.0 (May 19, 2023) +* **[#1109](https://github.com/awslabs/amazon-kinesis-client/pull/1109) Add support for stream ARNs** +* **[#1065](https://github.com/awslabs/amazon-kinesis-client/pull/1065) Allow tags to be added when lease table is created** +* [#1094](https://github.com/awslabs/amazon-kinesis-client/pull/1094) Code cleanup to introduce better testing +* [#1088](https://github.com/awslabs/amazon-kinesis-client/pull/1088) Minimize race in PSSM to optimize shard sync calls +* [#1086](https://github.com/awslabs/amazon-kinesis-client/pull/1086) Add additional SingleStreamTracker constructor with stream position parameter +* [#1084](https://github.com/awslabs/amazon-kinesis-client/pull/1084) More consistent testing behavior with restartAfterRequestTimerExpires +* [#1066](https://github.com/awslabs/amazon-kinesis-client/pull/1066) More consistent testing behavior with HashRangesAreAlwaysComplete +* [#1072](https://github.com/awslabs/amazon-kinesis-client/pull/1072) Upgrade nexus-staging-maven-plugin from 1.6.8 to 1.6.13 +* [#1073](https://github.com/awslabs/amazon-kinesis-client/pull/1073) Upgrade slf4j-api from 2.0.6 to 2.0.7 +* [#1090](https://github.com/awslabs/amazon-kinesis-client/pull/1090) Upgrade awssdk.version from 2.20.8 to 2.20.43 +* [#1071](https://github.com/awslabs/amazon-kinesis-client/pull/1071) Upgrade maven-compiler-plugin from 3.8.1 to 3.11.0 + ### Release 2.4.8 (March 21, 2023) * [#1080](https://github.com/awslabs/amazon-kinesis-client/pull/1080) Added metric in `ShutdownTask` for scenario when parent leases are missing. * [#1077](https://github.com/awslabs/amazon-kinesis-client/pull/1077) Reverted changes to pom property diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index db364375..0dfe7507 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.5.0-SNAPSHOT + 2.5.0 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index e08ed98d..29289509 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.5.0-SNAPSHOT + 2.5.0 amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 082dd565..9e9332b3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.0-SNAPSHOT"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = 2.5.0; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index 0f3e0ddf..b2c69a1f 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.5.0-SNAPSHOT + 2.5.0 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From a35c3a1599e4344254862c437621dd379d8fd2a9 Mon Sep 17 00:00:00 2001 From: furq-aws <127275086+furq-aws@users.noreply.github.com> Date: Fri, 19 May 2023 17:12:20 -0700 Subject: [PATCH 65/88] 2.5.0 Release Prep: Add quotations to RetrievalConfig KCLversion and remove CHANGELOG whitespace --- CHANGELOG.md | 2 +- .../java/software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1c494773..54a7892e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,7 +7,7 @@ For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/aws * **[#1109](https://github.com/awslabs/amazon-kinesis-client/pull/1109) Add support for stream ARNs** * **[#1065](https://github.com/awslabs/amazon-kinesis-client/pull/1065) Allow tags to be added when lease table is created** * [#1094](https://github.com/awslabs/amazon-kinesis-client/pull/1094) Code cleanup to introduce better testing -* [#1088](https://github.com/awslabs/amazon-kinesis-client/pull/1088) Minimize race in PSSM to optimize shard sync calls +* [#1088](https://github.com/awslabs/amazon-kinesis-client/pull/1088) Minimize race in PSSM to optimize shard sync calls * [#1086](https://github.com/awslabs/amazon-kinesis-client/pull/1086) Add additional SingleStreamTracker constructor with stream position parameter * [#1084](https://github.com/awslabs/amazon-kinesis-client/pull/1084) More consistent testing behavior with restartAfterRequestTimerExpires * [#1066](https://github.com/awslabs/amazon-kinesis-client/pull/1066) More consistent testing behavior with HashRangesAreAlwaysComplete diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 9e9332b3..a75fb377 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = 2.5.0; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.0"; /** * Client used to make calls to Kinesis for records retrieval From a8fc1367c6883eaf1bbb4a69b6bfac7423ae79c6 Mon Sep 17 00:00:00 2001 From: furq-aws <127275086+furq-aws@users.noreply.github.com> Date: Fri, 19 May 2023 19:45:31 -0700 Subject: [PATCH 66/88] Update KCL version to 2.5.1-SNAPSHOT (#1115) --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../java/software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 0dfe7507..b3a48210 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.5.0 + 2.5.1-SNAPSHOT 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 29289509..67c22703 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.5.0 + 2.5.1-SNAPSHOT amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index a75fb377..5d01edeb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.0"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.1-SNAPSHOT"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index b2c69a1f..7ebd6e54 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.5.0 + 2.5.1-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From f1ef0e820dda63e846687e49c9c10195ffc507cf Mon Sep 17 00:00:00 2001 From: mmankika-aws <134537845+mmankika-aws@users.noreply.github.com> Date: Tue, 20 Jun 2023 11:12:24 -0700 Subject: [PATCH 67/88] GitHub actions (#1145) * Adding Github Actions config --- .github/workflows/maven.yml | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 .github/workflows/maven.yml diff --git a/.github/workflows/maven.yml b/.github/workflows/maven.yml new file mode 100644 index 00000000..6bb4ea3c --- /dev/null +++ b/.github/workflows/maven.yml @@ -0,0 +1,32 @@ +# This workflow will build a Java project with Maven, and cache/restore any dependencies to improve the workflow execution time +# For more information see: https://docs.github.com/en/actions/automating-builds-and-tests/building-and-testing-java-with-maven + +# This workflow uses actions that are not certified by GitHub. +# They are provided by a third-party and are governed by +# separate terms of service, privacy policy, and support +# documentation. + +name: Java CI with Maven + +on: + push: + branches: + - "master" + pull_request: + branches: + - "master" + +jobs: + build: + + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v3 + - name: Set up JDK 8 + uses: actions/setup-java@v3 + with: + java-version: '8' + distribution: 'corretto' + - name: Build with Maven + run: mvn -B package --file pom.xml -DskipITs \ No newline at end of file From 53dbb4ea79eceea7dc8764448e87d71574bea0e0 Mon Sep 17 00:00:00 2001 From: mmankika-aws <134537845+mmankika-aws@users.noreply.github.com> Date: Wed, 21 Jun 2023 14:55:55 -0700 Subject: [PATCH 68/88] Adding testing architecture and KCL 2.x basic polling/streaming tests (#1136) * Adding testing architecture and KCL 2.x basic polling and streaming tests --- README.md | 8 +- amazon-kinesis-client/pom.xml | 4 + .../amazon/kinesis/config/KCLAppConfig.java | 192 +++++++++++++++ .../ReleaseCanaryPollingH1TestConfig.java | 41 ++++ .../ReleaseCanaryPollingH2TestConfig.java | 41 ++++ .../ReleaseCanaryStreamingTestConfig.java | 24 ++ .../BasicStreamConsumerIntegrationTest.java | 44 ++++ .../kinesis/utils/AWSResourceManager.java | 77 ++++++ .../kinesis/utils/LeaseTableManager.java | 68 ++++++ .../kinesis/utils/RecordValidationStatus.java | 10 + .../kinesis/utils/RecordValidatorQueue.java | 63 +++++ .../utils/RecordValidatorQueueTest.java | 44 ++++ .../amazon/kinesis/utils/ReshardOptions.java | 11 + .../kinesis/utils/StreamExistenceManager.java | 117 +++++++++ .../amazon/kinesis/utils/TestConsumer.java | 223 ++++++++++++++++++ .../kinesis/utils/TestRecordProcessor.java | 108 +++++++++ .../utils/TestRecordProcessorFactory.java | 19 ++ 17 files changed, 1093 insertions(+), 1 deletion(-) create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH1TestConfig.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH2TestConfig.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingTestConfig.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidationStatus.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueue.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueueTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java diff --git a/README.md b/README.md index b2ebf3ef..6328e115 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,13 @@ Please open an issue if you have any questions. After you've downloaded the code from GitHub, you can build it using Maven. To disable GPG signing in the build, use this command: `mvn clean install -Dgpg.skip=true`. Note: This command runs Integration tests, which in turn creates AWS resources (which requires manual cleanup). Integration tests require valid AWS credentials need to be discovered at - runtime. To skip running integration tests, add ` -DskipITs` option to the build command. + runtime. To skip running integration tests, add ` -DskipITs` option to the build command. + +## Running Integration Tests + +To run integration tests: `mvn -Dit.test=*IntegrationTest verify`. +This will look for a default AWS profile specified in your local `.aws/credentials`. +Optionally, you can provide the name of an IAM user/role to run tests with as a string using this command: `mvn -Dit.test=*IntegrationTest -DawsProfile="" verify`. ## Integration with the Kinesis Producer Library For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesis-guide-kpl]**, the KCL integrates without additional effort. When the KCL retrieves an aggregated Amazon Kinesis record consisting of multiple KPL user records, it will automatically invoke the KPL to extract the individual user records before returning them to the user. diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 67c22703..3eb8ad94 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -207,6 +207,10 @@ sqlite4java.library.path ${sqlite4java.libpath} + + awsProfile + ${awsProfile} + diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java new file mode 100644 index 00000000..b67efa10 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java @@ -0,0 +1,192 @@ +package software.amazon.kinesis.config; + +import lombok.Value; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.utils.RecordValidatorQueue; +import software.amazon.kinesis.utils.ReshardOptions; +import software.amazon.kinesis.utils.TestRecordProcessorFactory; +import lombok.Builder; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider; +import software.amazon.awssdk.http.Protocol; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClientBuilder; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClientBuilder; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClientBuilder; +import software.amazon.awssdk.utils.AttributeMap; +import software.amazon.kinesis.common.ConfigsBuilder; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.retrieval.RetrievalConfig; + +import java.io.IOException; +import java.net.Inet4Address; +import java.net.URISyntaxException; +import java.net.UnknownHostException; +import java.util.Optional; + +/** + * Default configuration for a producer or consumer used in integration tests. + * Producer: puts records of size 60 KB at an interval of 100 ms + * Consumer: streaming configuration (vs polling) that starts processing records at shard horizon + */ +public abstract class KCLAppConfig { + + private KinesisAsyncClient kinesisAsyncClient; + private DynamoDbAsyncClient dynamoDbAsyncClient; + private CloudWatchAsyncClient cloudWatchAsyncClient; + private RecordValidatorQueue recordValidator; + + /** + * Name used for test stream and lease tracker table + */ + public abstract String getStreamName(); + + public int getShardCount() { return 4; } + + public Region getRegion() { return Region.US_WEST_2; } + + /** + * "default" profile, should match with profiles listed in "cat ~/.aws/config" + */ + private AwsCredentialsProvider getCredentialsProvider() { + final String awsProfile = System.getProperty("awsProfile"); + return (awsProfile != null) ? + ProfileCredentialsProvider.builder().profileName(awsProfile).build() : DefaultCredentialsProvider.create(); + } + + public InitialPositionInStream getInitialPosition() { + return InitialPositionInStream.TRIM_HORIZON; + } + + public abstract Protocol getKinesisClientProtocol(); + + public ProducerConfig getProducerConfig() { + return ProducerConfig.builder() + .isBatchPut(false) + .batchSize(1) + .recordSizeKB(60) + .callPeriodMills(100) + .build(); + } + + public ReshardConfig getReshardConfig() { + return null; + } + + public final KinesisAsyncClient buildAsyncKinesisClient() throws URISyntaxException, IOException { + + if (kinesisAsyncClient == null) { + // Setup H2 client config. + final NettyNioAsyncHttpClient.Builder builder = NettyNioAsyncHttpClient.builder() + .maxConcurrency(Integer.MAX_VALUE); + + builder.protocol(getKinesisClientProtocol()); + + final SdkAsyncHttpClient sdkAsyncHttpClient = + builder.buildWithDefaults(AttributeMap.builder().build()); + + // Setup client builder by default values + final KinesisAsyncClientBuilder kinesisAsyncClientBuilder = KinesisAsyncClient.builder().region(getRegion()); + + kinesisAsyncClientBuilder.httpClient(sdkAsyncHttpClient); + + kinesisAsyncClientBuilder.credentialsProvider(getCredentialsProvider()); + + this.kinesisAsyncClient = kinesisAsyncClientBuilder.build(); + } + + return this.kinesisAsyncClient; + } + + public final DynamoDbAsyncClient buildAsyncDynamoDbClient() throws IOException { + if (this.dynamoDbAsyncClient == null) { + final DynamoDbAsyncClientBuilder builder = DynamoDbAsyncClient.builder().region(getRegion()); + builder.credentialsProvider(getCredentialsProvider()); + this.dynamoDbAsyncClient = builder.build(); + } + return this.dynamoDbAsyncClient; + } + + public final CloudWatchAsyncClient buildAsyncCloudWatchClient() throws IOException { + if (this.cloudWatchAsyncClient == null) { + final CloudWatchAsyncClientBuilder builder = CloudWatchAsyncClient.builder().region(getRegion()); + builder.credentialsProvider(getCredentialsProvider()); + this.cloudWatchAsyncClient = builder.build(); + } + return this.cloudWatchAsyncClient; + } + + public final String getWorkerId() throws UnknownHostException { + return Inet4Address.getLocalHost().getHostName(); + } + + public final RecordValidatorQueue getRecordValidator() { + if (recordValidator == null) { + this.recordValidator = new RecordValidatorQueue(); + } + return this.recordValidator; + } + + public ShardRecordProcessorFactory getShardRecordProcessorFactory() { + return new TestRecordProcessorFactory(getRecordValidator()); + } + + public final ConfigsBuilder getConfigsBuilder() throws IOException, URISyntaxException { + final String workerId = getWorkerId(); + return new ConfigsBuilder(getStreamName(), getStreamName(), buildAsyncKinesisClient(), buildAsyncDynamoDbClient(), + buildAsyncCloudWatchClient(), workerId, getShardRecordProcessorFactory()); + } + + public RetrievalConfig getRetrievalConfig() throws IOException, URISyntaxException { + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPosition(getInitialPosition()); + + // Default is a streaming consumer + final RetrievalConfig config = getConfigsBuilder().retrievalConfig(); + config.initialPositionInStreamExtended(initialPosition); + return config; + } + + /** + * Configure ingress load (batch size, record size, and calling interval) + */ + @Value + @Builder + static class ProducerConfig { + private boolean isBatchPut; + private int batchSize; + private int recordSizeKB; + private long callPeriodMills; + } + + /** + * Description of the method of resharding for a test case + */ + @Value + @Builder + static class ReshardConfig { + /** + * reshardingFactorCycle: lists the order or reshards that will be done during one reshard cycle + * e.g {SPLIT, MERGE} means that the number of shards will first be doubled, then halved + */ + private ReshardOptions[] reshardingFactorCycle; + + /** + * numReshardCycles: the number of resharding cycles that will be executed in a test + */ + private int numReshardCycles; + + /** + * reshardFrequencyMillis: the period of time between reshard cycles (in milliseconds) + */ + private long reshardFrequencyMillis; + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH1TestConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH1TestConfig.java new file mode 100644 index 00000000..07a29171 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH1TestConfig.java @@ -0,0 +1,41 @@ +package software.amazon.kinesis.config; + +import software.amazon.awssdk.http.Protocol; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.retrieval.RetrievalConfig; +import software.amazon.kinesis.retrieval.polling.PollingConfig; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.UUID; + +/** + * Config for a polling consumer with HTTP protocol of HTTP1 + */ +public class ReleaseCanaryPollingH1TestConfig extends KCLAppConfig { + + private final UUID uniqueId = UUID.randomUUID(); + + @Override + public String getStreamName() { + return "KCLReleaseCanary2XPollingH1TestStream_" + uniqueId; + } + + @Override + public Protocol getKinesisClientProtocol() { + return Protocol.HTTP1_1; + } + + @Override + public RetrievalConfig getRetrievalConfig() throws IOException, URISyntaxException { + + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPosition(getInitialPosition()); + + final RetrievalConfig config = getConfigsBuilder().retrievalConfig(); + config.initialPositionInStreamExtended(initialPosition); + config.retrievalSpecificConfig(new PollingConfig(getStreamName(), config.kinesisClient())); + + return config; + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH2TestConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH2TestConfig.java new file mode 100644 index 00000000..eb2b2143 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryPollingH2TestConfig.java @@ -0,0 +1,41 @@ +package software.amazon.kinesis.config; + +import software.amazon.awssdk.http.Protocol; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.retrieval.RetrievalConfig; +import software.amazon.kinesis.retrieval.polling.PollingConfig; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.UUID; + +/** + * Config for a polling consumer with HTTP protocol of HTTP2 + */ +public class ReleaseCanaryPollingH2TestConfig extends KCLAppConfig { + private final UUID uniqueId = UUID.randomUUID(); + + @Override + public String getStreamName() { + return "KCLReleaseCanary2XPollingH2TestStream_" + uniqueId; + } + + @Override + public Protocol getKinesisClientProtocol() { + return Protocol.HTTP2; + } + + @Override + public RetrievalConfig getRetrievalConfig() throws IOException, URISyntaxException { + + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPosition(getInitialPosition()); + + final RetrievalConfig config = getConfigsBuilder().retrievalConfig(); + config.initialPositionInStreamExtended(initialPosition); + config.retrievalSpecificConfig(new PollingConfig(getStreamName(), config.kinesisClient())); + + return config; + } +} + diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingTestConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingTestConfig.java new file mode 100644 index 00000000..6b0284c7 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingTestConfig.java @@ -0,0 +1,24 @@ +package software.amazon.kinesis.config; + +import software.amazon.awssdk.http.Protocol; + +import java.util.UUID; + +/** + * Config for a streaming consumer with HTTP protocol of HTTP2 + */ +public class ReleaseCanaryStreamingTestConfig extends KCLAppConfig { + private final UUID uniqueId = UUID.randomUUID(); + + @Override + public String getStreamName() { + return "KCLReleaseCanary2XStreamingTestStream_" + uniqueId; + } + + @Override + public Protocol getKinesisClientProtocol() { + return Protocol.HTTP2; + } + +} + diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java new file mode 100644 index 00000000..e2e44687 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java @@ -0,0 +1,44 @@ +package software.amazon.kinesis.lifecycle; + +import org.junit.Test; +import software.amazon.kinesis.config.KCLAppConfig; +import software.amazon.kinesis.config.ReleaseCanaryPollingH2TestConfig; +import software.amazon.kinesis.config.ReleaseCanaryPollingH1TestConfig; +import software.amazon.kinesis.config.ReleaseCanaryStreamingTestConfig; +import software.amazon.kinesis.utils.TestConsumer; + +public class BasicStreamConsumerIntegrationTest { + + /** + * Test with a polling consumer using HTTP2 protocol. + * In the polling case, consumer makes calls to the producer each time to request records to process. + */ + @Test + public void kclReleaseCanaryPollingH2Test() throws Exception { + KCLAppConfig consumerConfig = new ReleaseCanaryPollingH2TestConfig(); + TestConsumer consumer = new TestConsumer(consumerConfig); + consumer.run(); + } + + /** + * Test with a polling consumer using HTTP1 protocol. + * In the polling case, consumer makes calls to the producer each time to request records to process. + */ + @Test + public void kclReleaseCanaryPollingH1Test() throws Exception { + KCLAppConfig consumerConfig = new ReleaseCanaryPollingH1TestConfig(); + TestConsumer consumer = new TestConsumer(consumerConfig); + consumer.run(); + } + + /** + * Test with a streaming consumer. + * In the streaming configuration, connection is made once between consumer and producer and producer continuously sends data to be processed. + */ + @Test + public void kclReleaseCanaryStreamingTest() throws Exception { + KCLAppConfig consumerConfig = new ReleaseCanaryStreamingTestConfig(); + TestConsumer consumer = new TestConsumer(consumerConfig); + consumer.run(); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java new file mode 100644 index 00000000..3314f922 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java @@ -0,0 +1,77 @@ +package software.amazon.kinesis.utils; + +import lombok.NoArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.dynamodb.model.ListTablesRequest; +import software.amazon.awssdk.services.dynamodb.model.ListTablesResponse; +import software.amazon.kinesis.common.FutureUtils; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +@Slf4j +@NoArgsConstructor +public abstract class AWSResourceManager { + + /** + * Make delete resource API call for specific resource type + */ + public abstract void deleteResourceCall(String resourceName) throws Exception; + + /** + * Check if resource with given name is in active state + */ + public abstract boolean isResourceActive(String name); + + /** + * Get a list of all the names of resources of a specified type + * @return + * @throws Exception + */ + public abstract List getAllResourceNames() throws Exception; + + /** + * Delete resource with specified resource name + */ + public void deleteResource(String resourceName) throws Exception { + + try { + deleteResourceCall(resourceName); + } catch (Exception e) { + throw new Exception("Could not delete resource: {}", e); + } + + // Wait till resource is deleted to return + int i = 0; + while (true) { + i++; + if (i > 100) { + throw new RuntimeException("Failed resource deletion"); + } + try { + if (!isResourceActive(resourceName)) { + log.info("Successfully deleted the resource {}", resourceName); + return; + } + } catch (Exception e) { + try { + Thread.sleep(TimeUnit.SECONDS.toMillis(10)); + } catch (InterruptedException e1) {} + log.info("Resource {} is not deleted yet, exception: ", resourceName); + } + } + } + + /** + * Delete all instances of a particular resource type + */ + public void deleteAllResource() throws Exception { + final List resourceNames = getAllResourceNames(); + for (String resourceName : resourceNames) { + deleteResource(resourceName); + } + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java new file mode 100644 index 00000000..e8d1cb05 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java @@ -0,0 +1,68 @@ +package software.amazon.kinesis.utils; + +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.model.ListTablesRequest; +import software.amazon.awssdk.services.dynamodb.model.ListTablesResponse; +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; +import software.amazon.awssdk.services.dynamodb.model.TableStatus; +import software.amazon.kinesis.common.FutureUtils; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +@Slf4j +@AllArgsConstructor +public class LeaseTableManager extends AWSResourceManager { + + private final DynamoDbAsyncClient dynamoClient; + + public boolean isResourceActive(String tableName) { + final DescribeTableRequest request = DescribeTableRequest.builder().tableName(tableName).build(); + final CompletableFuture describeTableResponseCompletableFuture = dynamoClient.describeTable(request); + + try { + final DescribeTableResponse response = describeTableResponseCompletableFuture.get(30, TimeUnit.SECONDS); + boolean isActive = response.table().tableStatus().equals(TableStatus.ACTIVE); + if (!isActive) { + throw new RuntimeException("Table is not active, instead in status: " + response.table().tableStatus()); + } + return true; + } catch (ExecutionException e) { + if (e.getCause() instanceof ResourceNotFoundException) { + return false; + } else { + throw new RuntimeException(e); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void deleteResourceCall(String tableName) throws Exception { + final DeleteTableRequest request = DeleteTableRequest.builder().tableName(tableName).build(); + FutureUtils.resolveOrCancelFuture(dynamoClient.deleteTable(request), Duration.ofSeconds(60)); + } + + public List getAllResourceNames() throws Exception { + ListTablesRequest listTableRequest = ListTablesRequest.builder().build(); + List allTableNames = new ArrayList<>(); + ListTablesResponse result = null; + do { + result = FutureUtils.resolveOrCancelFuture(dynamoClient.listTables(listTableRequest), Duration.ofSeconds(60)); + allTableNames.addAll(result.tableNames()); + listTableRequest = ListTablesRequest.builder().exclusiveStartTableName(result.lastEvaluatedTableName()).build(); + } while (result.lastEvaluatedTableName() != null); + return allTableNames; + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidationStatus.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidationStatus.java new file mode 100644 index 00000000..998d5c44 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidationStatus.java @@ -0,0 +1,10 @@ +package software.amazon.kinesis.utils; + +/** + * Possible outcomes for record validation in RecordValidatorQueue + */ +public enum RecordValidationStatus { + OUT_OF_ORDER, + MISSING_RECORD, + NO_ERROR +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueue.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueue.java new file mode 100644 index 00000000..c6b9d6a6 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueue.java @@ -0,0 +1,63 @@ +package software.amazon.kinesis.utils; + +import lombok.extern.slf4j.Slf4j; + +import java.util.HashSet; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Class that maintains a dictionary that maps shard IDs to a list of records + * that are processed by that shard. + * Validation ensures that + * 1. The records processed by each shard are in increasing order (duplicates allowed) + * 2. The total number of unique records processed is equal to the number of records put on the stream + */ +@Slf4j +public class RecordValidatorQueue { + + private final ConcurrentHashMap> dict = new ConcurrentHashMap<>(); + + public void add(String shardId, String data) { + final List values = dict.computeIfAbsent(shardId, key -> new ArrayList<>()); + values.add(data); + } + + public RecordValidationStatus validateRecords(int expectedRecordCount) { + + // Validate that each List in the HashMap has data records in increasing order + for (Map.Entry> entry : dict.entrySet()) { + List recordsPerShard = entry.getValue(); + int prevVal = -1; + for (String record : recordsPerShard) { + int nextVal = Integer.parseInt(record); + if (prevVal > nextVal) { + log.error("The records are not in increasing order. Saw record data {} before {}.", prevVal, nextVal); + return RecordValidationStatus.OUT_OF_ORDER; + } + prevVal = nextVal; + } + } + + // Validate that no records are missing over all shards + int actualRecordCount = 0; + for (Map.Entry> entry : dict.entrySet()) { + List recordsPerShard = entry.getValue(); + Set noDupRecords = new HashSet(recordsPerShard); + actualRecordCount += noDupRecords.size(); + } + + // If this is true, then there was some record that was missed during processing. + if (actualRecordCount != expectedRecordCount) { + log.error("Failed to get correct number of records processed. Should be {} but was {}", expectedRecordCount, actualRecordCount); + return RecordValidationStatus.MISSING_RECORD; + } + + // Record validation succeeded. + return RecordValidationStatus.NO_ERROR; + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueueTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueueTest.java new file mode 100644 index 00000000..c196aa54 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/RecordValidatorQueueTest.java @@ -0,0 +1,44 @@ +package software.amazon.kinesis.utils; + +import org.junit.Assert; +import org.junit.Test; + +public class RecordValidatorQueueTest { + + private final RecordValidatorQueue recordValidator = new RecordValidatorQueue(); + + private static final String SHARD_ID = "ABC"; + + @Test + public void testValidationFailedRecordOutOfOrder() { + recordValidator.add(SHARD_ID, "0"); + recordValidator.add(SHARD_ID, "1"); + recordValidator.add(SHARD_ID, "3"); + recordValidator.add(SHARD_ID, "2"); + + RecordValidationStatus error = recordValidator.validateRecords(4); + Assert.assertEquals(RecordValidationStatus.OUT_OF_ORDER, error); + } + + @Test + public void testValidationFailedMissingRecord() { + recordValidator.add(SHARD_ID, "0"); + recordValidator.add(SHARD_ID, "1"); + recordValidator.add(SHARD_ID, "2"); + recordValidator.add(SHARD_ID, "3"); + + RecordValidationStatus error = recordValidator.validateRecords(5); + Assert.assertEquals(RecordValidationStatus.MISSING_RECORD, error); + } + + @Test + public void testValidRecords() { + recordValidator.add(SHARD_ID, "0"); + recordValidator.add(SHARD_ID, "1"); + recordValidator.add(SHARD_ID, "2"); + recordValidator.add(SHARD_ID, "3"); + + RecordValidationStatus error = recordValidator.validateRecords(4); + Assert.assertEquals(RecordValidationStatus.NO_ERROR, error); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java new file mode 100644 index 00000000..fbf5f68b --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java @@ -0,0 +1,11 @@ +package software.amazon.kinesis.utils; + +/** + * Specifies the types of resharding possible in integration tests + * Split doubles the number of shards. + * Merge halves the number of shards. + */ +public enum ReshardOptions { + SPLIT, + MERGE +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java new file mode 100644 index 00000000..eeffb36b --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java @@ -0,0 +1,117 @@ +package software.amazon.kinesis.utils; + +import lombok.Value; +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.dynamodb.model.ListTablesRequest; +import software.amazon.awssdk.services.dynamodb.model.ListTablesResponse; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DeleteStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; +import software.amazon.awssdk.services.kinesis.model.ListStreamsRequest; +import software.amazon.awssdk.services.kinesis.model.ListStreamsResponse; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; +import software.amazon.awssdk.services.kinesis.model.StreamStatus; +import software.amazon.kinesis.common.FutureUtils; +import software.amazon.kinesis.config.KCLAppConfig; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +@Value +@Slf4j +public class StreamExistenceManager extends AWSResourceManager { + private final KinesisAsyncClient client; + private final KCLAppConfig testConfig; + + public StreamExistenceManager(KCLAppConfig config) throws URISyntaxException, IOException { + this.testConfig = config; + this.client = config.buildAsyncKinesisClient(); + } + + public boolean isResourceActive(String streamName) { + final DescribeStreamSummaryRequest request = DescribeStreamSummaryRequest.builder().streamName(streamName).build(); + final CompletableFuture describeStreamSummaryResponseCompletableFuture = client.describeStreamSummary(request); + + try { + final DescribeStreamSummaryResponse response = describeStreamSummaryResponseCompletableFuture.get(30, TimeUnit.SECONDS); + boolean isActive = response.streamDescriptionSummary().streamStatus().equals(StreamStatus.ACTIVE); + if (!isActive) { + throw new RuntimeException("Stream is not active, instead in status: " + response.streamDescriptionSummary().streamStatus()); + } + return true; + } catch (ExecutionException e) { + if (e.getCause() instanceof ResourceNotFoundException) { + return false; + } else { + throw new RuntimeException(e); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void deleteResourceCall(String streamName) throws Exception { + final DeleteStreamRequest request = DeleteStreamRequest.builder().streamName(streamName).enforceConsumerDeletion(true).build(); + client.deleteStream(request).get(30, TimeUnit.SECONDS); + } + + public List getAllResourceNames() throws Exception { + ListStreamsRequest listStreamRequest = ListStreamsRequest.builder().build(); + List allStreamNames = new ArrayList<>(); + ListStreamsResponse result = null; + do { + result = FutureUtils.resolveOrCancelFuture(client.listStreams(listStreamRequest), Duration.ofSeconds(60)); + allStreamNames.addAll(result.streamNames()); + listStreamRequest = ListStreamsRequest.builder().exclusiveStartStreamName(result.nextToken()).build(); + } while (result.hasMoreStreams()); + return allStreamNames; + } + + public void checkStreamAndCreateIfNecessary(String streamName) { + + if (!isResourceActive(streamName)) { + createStream(streamName, testConfig.getShardCount()); + } + log.info("Using stream {} with region {}", streamName, testConfig.getRegion()); + } + + private void createStream(String streamName, int shardCount) { + final CreateStreamRequest request = CreateStreamRequest.builder().streamName(streamName).shardCount(shardCount).build(); + try { + client.createStream(request).get(30, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException("Failed to create stream with name " + streamName, e); + } + + int i = 0; + while (true) { + i++; + if (i > 100) { + throw new RuntimeException("Failed stream creation, did not transition into active"); + } + try { + boolean isActive = isResourceActive(streamName); + if (isActive) { + log.info("Succesfully created the stream {}", streamName); + return; + } + } catch (Exception e) { + try { + Thread.sleep(TimeUnit.SECONDS.toMillis(10)); + } catch (InterruptedException e1) { + log.error("Failed to sleep"); + } + log.info("Stream {} is not active yet, exception: ", streamName, e); + } + } + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java new file mode 100644 index 00000000..223ca99a --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java @@ -0,0 +1,223 @@ +package software.amazon.kinesis.utils; + +import com.fasterxml.jackson.databind.ObjectMapper; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.RandomStringUtils; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.PutRecordRequest; +import software.amazon.kinesis.checkpoint.CheckpointConfig; +import software.amazon.kinesis.common.ConfigsBuilder; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.config.KCLAppConfig; +import software.amazon.kinesis.coordinator.CoordinatorConfig; +import software.amazon.kinesis.coordinator.Scheduler; +import software.amazon.kinesis.leases.LeaseManagementConfig; +import software.amazon.kinesis.lifecycle.LifecycleConfig; +import software.amazon.kinesis.metrics.MetricsConfig; +import software.amazon.kinesis.processor.ProcessorConfig; +import software.amazon.kinesis.retrieval.RetrievalConfig; + +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +@Slf4j +public class TestConsumer { + public final KCLAppConfig consumerConfig; + public final Region region; + public final String streamName; + public final KinesisAsyncClient kinesisClient; + private MetricsConfig metricsConfig; + private RetrievalConfig retrievalConfig; + private CheckpointConfig checkpointConfig; + private CoordinatorConfig coordinatorConfig; + private LeaseManagementConfig leaseManagementConfig; + private LifecycleConfig lifecycleConfig; + private ProcessorConfig processorConfig; + private Scheduler scheduler; + private ScheduledExecutorService producerExecutor; + private ScheduledFuture producerFuture; + private ScheduledExecutorService consumerExecutor; + private ScheduledFuture consumerFuture; + private DynamoDbAsyncClient dynamoClient; + private final ObjectMapper mapper = new ObjectMapper(); + public int successfulPutRecords = 0; + public BigInteger payloadCounter = new BigInteger("0"); + + public TestConsumer(KCLAppConfig consumerConfig) throws Exception { + this.consumerConfig = consumerConfig; + this.region = consumerConfig.getRegion(); + this.streamName = consumerConfig.getStreamName(); + this.kinesisClient = consumerConfig.buildAsyncKinesisClient(); + this.dynamoClient = consumerConfig.buildAsyncDynamoDbClient(); + } + + public void run() throws Exception { + + final StreamExistenceManager streamExistenceManager = new StreamExistenceManager(this.consumerConfig); + final LeaseTableManager leaseTableManager = new LeaseTableManager(this.dynamoClient); + + // Clean up any old streams or lease tables left in test environment + cleanTestResources(streamExistenceManager, leaseTableManager); + + // Check if stream is created. If not, create it + streamExistenceManager.checkStreamAndCreateIfNecessary(this.streamName); + + startProducer(); + setUpConsumerResources(); + + try { + startConsumer(); + + // Sleep for three minutes to allow the producer/consumer to run and then end the test case. + Thread.sleep(TimeUnit.SECONDS.toMillis(60 * 3)); + + // Stops sending dummy data. + stopProducer(); + + // Wait a few seconds for the last few records to be processed + Thread.sleep(TimeUnit.SECONDS.toMillis(10)); + + // Finishes processing current batch of data already received from Kinesis before shutting down. + awaitConsumerFinish(); + + // Validate processed data + validateRecordProcessor(); + + } catch (Exception e) { + // Test Failed. Clean up resources and then throw exception. + log.info("----------Test Failed: Cleaning up resources------------"); + throw e; + } finally { + // Clean up resources created + deleteResources(streamExistenceManager, leaseTableManager); + } + } + + private void cleanTestResources(StreamExistenceManager streamExistenceManager, LeaseTableManager leaseTableManager) throws Exception { + log.info("----------Before starting, Cleaning test environment----------"); + log.info("----------Deleting all lease tables in account----------"); + leaseTableManager.deleteAllResource(); + log.info("----------Finished deleting all lease tables-------------"); + + log.info("----------Deleting all streams in account----------"); + streamExistenceManager.deleteAllResource(); + log.info("----------Finished deleting all streams-------------"); + } + + private void startProducer() { + // Send dummy data to stream + this.producerExecutor = Executors.newSingleThreadScheduledExecutor(); + this.producerFuture = producerExecutor.scheduleAtFixedRate(this::publishRecord, 60, 1, TimeUnit.SECONDS); + } + + private void setUpConsumerResources() throws Exception { + // Setup configuration of KCL (including DynamoDB and CloudWatch) + final ConfigsBuilder configsBuilder = consumerConfig.getConfigsBuilder(); + + retrievalConfig = consumerConfig.getRetrievalConfig(); + checkpointConfig = configsBuilder.checkpointConfig(); + coordinatorConfig = configsBuilder.coordinatorConfig(); + leaseManagementConfig = configsBuilder.leaseManagementConfig() + .initialPositionInStream(InitialPositionInStreamExtended.newInitialPosition(consumerConfig.getInitialPosition())) + .initialLeaseTableReadCapacity(50).initialLeaseTableWriteCapacity(50); + lifecycleConfig = configsBuilder.lifecycleConfig(); + processorConfig = configsBuilder.processorConfig(); + metricsConfig = configsBuilder.metricsConfig(); + + // Create Scheduler + this.scheduler = new Scheduler( + checkpointConfig, + coordinatorConfig, + leaseManagementConfig, + lifecycleConfig, + metricsConfig, + processorConfig, + retrievalConfig + ); + } + + private void startConsumer() { + // Start record processing of dummy data + this.consumerExecutor = Executors.newSingleThreadScheduledExecutor(); + this.consumerFuture = consumerExecutor.schedule(scheduler, 0, TimeUnit.SECONDS); + } + + public void publishRecord() { + final PutRecordRequest request; + try { + request = PutRecordRequest.builder() + .partitionKey(RandomStringUtils.randomAlphabetic(5, 20)) + .streamName(this.streamName) + .data(SdkBytes.fromByteBuffer(wrapWithCounter(5, payloadCounter))) // 1024 is 1 KB + .build(); + kinesisClient.putRecord(request).get(); + + // Increment the payload counter if the putRecord call was successful + payloadCounter = payloadCounter.add(new BigInteger("1")); + successfulPutRecords += 1; + log.info("---------Record published, successfulPutRecords is now: {}", successfulPutRecords); + } catch (InterruptedException e) { + log.info("Interrupted, assuming shutdown. ", e); + } catch (ExecutionException | RuntimeException e) { + log.error("Error during publish records", e); + } + } + + private ByteBuffer wrapWithCounter(int payloadSize, BigInteger payloadCounter) throws RuntimeException { + final byte[] returnData; + log.info("--------------Putting record with data: {}", payloadCounter); + try { + returnData = mapper.writeValueAsBytes(payloadCounter); + } catch (Exception e) { + throw new RuntimeException("Error converting object to bytes: ", e); + } + return ByteBuffer.wrap(returnData); + } + + private void stopProducer() { + log.info("Cancelling producer and shutting down executor."); + producerFuture.cancel(false); + producerExecutor.shutdown(); + } + + private void awaitConsumerFinish() throws Exception { + Future gracefulShutdownFuture = scheduler.startGracefulShutdown(); + log.info("Waiting up to 20 seconds for shutdown to complete."); + try { + gracefulShutdownFuture.get(20, TimeUnit.SECONDS); + } catch (InterruptedException e) { + log.info("Interrupted while waiting for graceful shutdown. Continuing."); + } catch (ExecutionException | TimeoutException e) { + throw e; + } + log.info("Completed, shutting down now."); + } + + private void validateRecordProcessor() throws Exception { + log.info("The number of expected records is: {}", successfulPutRecords); + final RecordValidationStatus errorVal = consumerConfig.getRecordValidator().validateRecords(successfulPutRecords); + if (errorVal != RecordValidationStatus.NO_ERROR) { + throw new RuntimeException("There was an error validating the records that were processed: " + errorVal.toString()); + } + log.info("--------------Completed validation of processed records.--------------"); + } + + private void deleteResources(StreamExistenceManager streamExistenceManager, LeaseTableManager leaseTableManager) throws Exception { + log.info("-------------Start deleting stream.----------------"); + streamExistenceManager.deleteResource(this.streamName); + log.info("-------------Start deleting lease table.----------------"); + leaseTableManager.deleteResource(this.consumerConfig.getStreamName()); + log.info("-------------Finished deleting resources.----------------"); + } + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java new file mode 100644 index 00000000..f3e43915 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java @@ -0,0 +1,108 @@ +package software.amazon.kinesis.utils; + +import lombok.extern.slf4j.Slf4j; +import org.slf4j.MDC; +import software.amazon.kinesis.exceptions.InvalidStateException; +import software.amazon.kinesis.exceptions.ShutdownException; +import software.amazon.kinesis.lifecycle.events.LeaseLostInput; +import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; +import software.amazon.kinesis.lifecycle.events.ShardEndedInput; +import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; +import software.amazon.kinesis.processor.ShardRecordProcessor; +import software.amazon.kinesis.lifecycle.events.InitializationInput; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.nio.ByteBuffer; + +/** + * Implement initialization and deletion of shards and shard record processing + */ +@Slf4j +public class TestRecordProcessor implements ShardRecordProcessor { + + private static final String SHARD_ID_MDC_KEY = "ShardId"; + + private String shardId; + + private final RecordValidatorQueue recordValidator; + + public TestRecordProcessor(RecordValidatorQueue recordValidator) { + this.recordValidator = recordValidator; + } + + @Override + public void initialize(InitializationInput initializationInput) { + shardId = initializationInput.shardId(); + MDC.put(SHARD_ID_MDC_KEY, shardId); + try { + log.info("Initializing @ Sequence: {}", initializationInput.extendedSequenceNumber()); + } finally { + MDC.remove(SHARD_ID_MDC_KEY); + } + } + + + @Override + public void processRecords(ProcessRecordsInput processRecordsInput) { + MDC.put(SHARD_ID_MDC_KEY, shardId); + try { + log.info("Processing {} record(s)", processRecordsInput.records().size()); + + for (KinesisClientRecord kinesisRecord : processRecordsInput.records()) { + final String data = new String(asByteArray(kinesisRecord.data())); + log.info("Processing record pk: {}", data); + recordValidator.add(shardId, data); + } + + } catch (Throwable t) { + log.error("Caught throwable while processing records. Aborting.", t); + Runtime.getRuntime().halt(1); + } finally { + MDC.remove(SHARD_ID_MDC_KEY); + } + } + + public static byte[] asByteArray(ByteBuffer buf) { + byte[] bytes = new byte[buf.remaining()]; + buf.get(bytes); + return bytes; + } + + @Override + public void leaseLost(LeaseLostInput leaseLostInput) { + MDC.put(SHARD_ID_MDC_KEY, shardId); + try { + log.info("Lost lease, so terminating."); + } finally { + MDC.remove(SHARD_ID_MDC_KEY); + } + } + + @Override + public void shardEnded(ShardEndedInput shardEndedInput) { + MDC.put(SHARD_ID_MDC_KEY, shardId); + try { + log.info("Reached shard end checkpointing."); + shardEndedInput.checkpointer().checkpoint(); + } catch (ShutdownException | InvalidStateException e) { + log.error("Exception while checkpointing at shard end. Giving up.", e); + } finally { + MDC.remove(SHARD_ID_MDC_KEY); + } + } + + @Override + public void shutdownRequested(ShutdownRequestedInput shutdownRequestedInput) { + MDC.put(SHARD_ID_MDC_KEY, shardId); + try { + log.info("Scheduler is shutting down, checkpointing."); + shutdownRequestedInput.checkpointer().checkpoint(); + } catch (ShutdownException | InvalidStateException e) { + log.error("Exception while checkpointing at requested shutdown. Giving up.", e); + } finally { + MDC.remove(SHARD_ID_MDC_KEY); + } + } + + +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java new file mode 100644 index 00000000..03361b6e --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java @@ -0,0 +1,19 @@ +package software.amazon.kinesis.utils; + +import software.amazon.kinesis.processor.ShardRecordProcessor; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; + +public class TestRecordProcessorFactory implements ShardRecordProcessorFactory { + + private final RecordValidatorQueue recordValidator; + + public TestRecordProcessorFactory(RecordValidatorQueue recordValidator) { + this.recordValidator = recordValidator; + } + + @Override + public ShardRecordProcessor shardRecordProcessor() { + return new TestRecordProcessor(this.recordValidator); + } + +} From 3d6800874cdc5e4c18df6ea0197f607f6298cab7 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Fri, 23 Jun 2023 14:58:10 -0400 Subject: [PATCH 69/88] Code cleanup to faciliate Checkstyle enforcement. (#1148) No functional change. --- .../ShardRecordProcessorCheckpointer.java | 6 ++-- .../dynamodb/DynamoDBCheckpointer.java | 3 +- .../kinesis/common/HashKeyRangeForLease.java | 3 +- .../NoOpWorkerStateChangeListener.java | 16 +++++----- .../coordinator/PeriodicShardSyncManager.java | 7 +++-- .../amazon/kinesis/coordinator/Scheduler.java | 5 +-- .../WorkerStateChangeListener.java | 20 ++++++------ .../leases/HierarchicalShardSyncer.java | 31 +++++++++++-------- .../kinesis/leases/LeaseCleanupManager.java | 2 +- .../kinesis/leases/LeaseManagementConfig.java | 4 +-- .../amazon/kinesis/leases/ShardSyncTask.java | 2 +- .../dynamodb/DynamoDBLeaseRefresher.java | 4 +-- .../dynamodb/DynamoDBLeaseSerializer.java | 4 +-- .../CustomerApplicationException.java | 6 ++-- .../amazon/kinesis/lifecycle/ProcessTask.java | 6 ++-- .../lifecycle/ShardConsumerSubscriber.java | 3 +- .../kinesis/lifecycle/ShutdownTask.java | 2 +- .../kinesis/metrics/CloudWatchMetricKey.java | 13 ++++---- .../kinesis/metrics/MetricDatumWithKey.java | 10 +++--- .../fanout/FanOutRecordsPublisher.java | 23 +++++++++----- .../fanout/FanOutRetrievalFactory.java | 2 +- .../retrieval/polling/PollingConfig.java | 2 +- .../polling/PrefetchRecordsPublisher.java | 4 +-- 23 files changed, 101 insertions(+), 77 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java index fd375264..5fbac1d7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java @@ -144,7 +144,8 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi * {@inheritDoc} */ @Override - public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { return prepareCheckpoint(largestPermittedCheckpointValue.sequenceNumber(), applicationState); } @@ -152,7 +153,8 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi * {@inheritDoc} */ @Override - public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { // // TODO: UserRecord Deprecation // diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java index d9646351..1aa258bb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java @@ -103,7 +103,8 @@ public class DynamoDBCheckpointer implements Checkpointer { } @Override - public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException { + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, + byte[] pendingCheckpointState) throws KinesisClientLibException { try { boolean wasSuccessful = prepareCheckpoint(leaseKey, pendingCheckpoint, UUID.fromString(concurrencyToken), pendingCheckpointState); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java index d2540073..30f8963a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java @@ -23,10 +23,11 @@ import software.amazon.awssdk.services.kinesis.model.HashKeyRange; import java.math.BigInteger; -@Value @Accessors(fluent = true) /** * Lease POJO to hold the starting hashkey range and ending hashkey range of kinesis shards. */ +@Accessors(fluent = true) +@Value public class HashKeyRangeForLease { private final BigInteger startingHashKey; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/NoOpWorkerStateChangeListener.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/NoOpWorkerStateChangeListener.java index 3e0432cb..ec21e4f6 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/NoOpWorkerStateChangeListener.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/NoOpWorkerStateChangeListener.java @@ -16,15 +16,15 @@ package software.amazon.kinesis.coordinator; public class NoOpWorkerStateChangeListener implements WorkerStateChangeListener { - /** - * Empty constructor for NoOp Worker State Change Listener - */ - public NoOpWorkerStateChangeListener() { + /** + * Empty constructor for NoOp Worker State Change Listener + */ + public NoOpWorkerStateChangeListener() { - } + } - @Override - public void onWorkerStateChange(WorkerState newState) { + @Override + public void onWorkerStateChange(WorkerState newState) { - } + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java index dac77351..aecb1331 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java @@ -348,7 +348,7 @@ class PeriodicShardSyncManager { ((MultiStreamLease) lease).shardId() : lease.leaseKey(); final Shard shard = kinesisShards.get(shardId); - if(shard == null) { + if (shard == null) { return lease; } lease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); @@ -372,7 +372,7 @@ class PeriodicShardSyncManager { List leasesWithHashKeyRanges) { // Sort the hash ranges by starting hash key. List sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges); - if(sortedLeasesWithHashKeyRanges.isEmpty()) { + if (sortedLeasesWithHashKeyRanges.isEmpty()) { log.error("No leases with valid hashranges found for stream {}", streamIdentifier); return Optional.of(new HashRangeHole()); } @@ -417,8 +417,9 @@ class PeriodicShardSyncManager { @VisibleForTesting static List sortLeasesByHashRange(List leasesWithHashKeyRanges) { - if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1) + if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1) { return leasesWithHashKeyRanges; + } Collections.sort(leasesWithHashKeyRanges, new HashKeyRangeComparator()); return leasesWithHashKeyRanges; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 49d271b2..d67fff96 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -544,7 +544,8 @@ public class Scheduler implements Runnable { final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors .partitioningBy(newStreamConfigMap::containsKey, Collectors.toSet())); final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier -> - Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); + Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()) + .collect(Collectors.toSet()); // These are the streams which are deleted in Kinesis and we encounter resource not found during // shardSyncTask. This is applicable in MultiStreamMode only, in case of SingleStreamMode, store will // not have any data. @@ -611,7 +612,7 @@ public class Scheduler implements Runnable { } private void removeStreamsFromStaleStreamsList(Set streamIdentifiers) { - for(StreamIdentifier streamIdentifier : streamIdentifiers) { + for (StreamIdentifier streamIdentifier : streamIdentifiers) { staleStreamDeletionMap.remove(streamIdentifier); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/WorkerStateChangeListener.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/WorkerStateChangeListener.java index ddce2a10..dd7162b3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/WorkerStateChangeListener.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/WorkerStateChangeListener.java @@ -19,16 +19,16 @@ package software.amazon.kinesis.coordinator; */ @FunctionalInterface public interface WorkerStateChangeListener { - enum WorkerState { - CREATED, - INITIALIZING, - STARTED, - SHUT_DOWN_STARTED, - SHUT_DOWN - } + enum WorkerState { + CREATED, + INITIALIZING, + STARTED, + SHUT_DOWN_STARTED, + SHUT_DOWN + } - void onWorkerStateChange(WorkerState newState); + void onWorkerStateChange(WorkerState newState); - default void onAllInitializationAttemptsFailed(Throwable e) { - } + default void onAllInitializationAttemptsFailed(Throwable e) { + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index b71796d3..534b5fd3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -80,7 +80,7 @@ public class HierarchicalShardSyncer { private static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); private static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); - private static final int retriesForCompleteHashRange = 3; + private static final int RETRIES_FOR_COMPLETE_HASH_RANGE = 3; private static final long DELAY_BETWEEN_LIST_SHARDS_MILLIS = 1000; @@ -98,7 +98,7 @@ public class HierarchicalShardSyncer { this.deletedStreamListProvider = deletedStreamListProvider; } - private static final BiFunction shardIdFromLeaseDeducer = + private static final BiFunction SHARD_ID_FROM_LEASE_DEDUCER = (lease, multiStreamArgs) -> multiStreamArgs.isMultiStreamMode() ? ((MultiStreamLease) lease).shardId() : @@ -129,7 +129,9 @@ public class HierarchicalShardSyncer { isLeaseTableEmpty); } - //Provide a pre-collcted list of shards to avoid calling ListShards API + /** + * Provide a pre-collected list of shards to avoid calling ListShards API + */ public synchronized boolean checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, List latestShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty) @@ -163,7 +165,7 @@ public class HierarchicalShardSyncer { final long startTime = System.currentTimeMillis(); boolean success = false; try { - if(leaseRefresher.createLeaseIfNotExists(lease)) { + if (leaseRefresher.createLeaseIfNotExists(lease)) { createdLeases.add(lease); } success = true; @@ -268,7 +270,7 @@ public class HierarchicalShardSyncer { List shards; - for (int i = 0; i < retriesForCompleteHashRange; i++) { + for (int i = 0; i < RETRIES_FOR_COMPLETE_HASH_RANGE; i++) { shards = shardDetector.listShardsWithFilter(shardFilter); if (shards == null) { @@ -284,7 +286,7 @@ public class HierarchicalShardSyncer { } throw new KinesisClientLibIOException("Hash range of shards returned for " + streamName + " was incomplete after " - + retriesForCompleteHashRange + " retries."); + + RETRIES_FOR_COMPLETE_HASH_RANGE + " retries."); } private List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { @@ -365,7 +367,8 @@ public class HierarchicalShardSyncer { * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, - final List currentLeases, final InitialPositionInStreamExtended initialPosition,final Set inconsistentShardIds) { + final List currentLeases, final InitialPositionInStreamExtended initialPosition, + final Set inconsistentShardIds) { return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds, new MultiStreamArgs(false, null)); } @@ -499,11 +502,13 @@ public class HierarchicalShardSyncer { if (descendantParentShardIds.contains(parentShardId) && !initialPosition.getInitialPositionInStream() .equals(InitialPositionInStream.AT_TIMESTAMP)) { - log.info("Setting Lease '{}' checkpoint to 'TRIM_HORIZON'. Checkpoint was previously set to {}", lease.leaseKey(), lease.checkpoint()); + log.info("Setting Lease '{}' checkpoint to 'TRIM_HORIZON'. Checkpoint was previously set to {}", + lease.leaseKey(), lease.checkpoint()); lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); } else { final ExtendedSequenceNumber newCheckpoint = convertToCheckpoint(initialPosition); - log.info("Setting Lease '{}' checkpoint to '{}'. Checkpoint was previously set to {}", lease.leaseKey(), newCheckpoint, lease.checkpoint()); + log.info("Setting Lease '{}' checkpoint to '{}'. Checkpoint was previously set to {}", + lease.leaseKey(), newCheckpoint, lease.checkpoint()); lease.checkpoint(newCheckpoint); } } @@ -728,8 +733,8 @@ public class HierarchicalShardSyncer { @Override public int compare(final Lease lease1, final Lease lease2) { int result = 0; - final String shardId1 = shardIdFromLeaseDeducer.apply(lease1, multiStreamArgs); - final String shardId2 = shardIdFromLeaseDeducer.apply(lease2, multiStreamArgs); + final String shardId1 = SHARD_ID_FROM_LEASE_DEDUCER.apply(lease1, multiStreamArgs); + final String shardId2 = SHARD_ID_FROM_LEASE_DEDUCER.apply(lease2, multiStreamArgs); final Shard shard1 = shardIdToShardMap.get(shardId1); final Shard shard2 = shardIdToShardMap.get(shardId2); @@ -802,7 +807,7 @@ public class HierarchicalShardSyncer { final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); currentLeases.stream().peek(lease -> log.debug("{} : Existing lease: {}", streamIdentifier, lease)) - .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .map(lease -> SHARD_ID_FROM_LEASE_DEDUCER.apply(lease, multiStreamArgs)) .collect(Collectors.toSet()); final List newLeasesToCreate = getLeasesToCreateForOpenAndClosedShards(initialPosition, shards, multiStreamArgs, streamIdentifier); @@ -908,7 +913,7 @@ public class HierarchicalShardSyncer { .map(streamId -> streamId.serialize()).orElse(""); final Set shardIdsOfCurrentLeases = currentLeases.stream() .peek(lease -> log.debug("{} : Existing lease: {}", streamIdentifier, lease)) - .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .map(lease -> SHARD_ID_FROM_LEASE_DEDUCER.apply(lease, multiStreamArgs)) .collect(Collectors.toSet()); final List openShards = getOpenShards(shards, streamIdentifier); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java index d62cd476..8a442bd3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java @@ -179,7 +179,7 @@ public class LeaseCleanupManager { try { if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) { final Lease leaseFromDDB = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey()); - if(leaseFromDDB != null) { + if (leaseFromDDB != null) { Set childShardKeys = leaseFromDDB.childShardIds(); if (CollectionUtils.isNullOrEmpty(childShardKeys)) { try { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java index d80799fa..9b95f562 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java @@ -310,7 +310,7 @@ public class LeaseManagementConfig { private LeaseManagementFactory leaseManagementFactory; public HierarchicalShardSyncer hierarchicalShardSyncer() { - if(hierarchicalShardSyncer == null) { + if (hierarchicalShardSyncer == null) { hierarchicalShardSyncer = new HierarchicalShardSyncer(); } return hierarchicalShardSyncer; @@ -356,7 +356,7 @@ public class LeaseManagementConfig { * @return LeaseManagementFactory */ public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer, boolean isMultiStreamingMode) { - if(leaseManagementFactory == null) { + if (leaseManagementFactory == null) { leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), dynamoDBClient(), tableName(), diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java index dd576114..1986fa49 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java @@ -37,7 +37,7 @@ import software.amazon.kinesis.metrics.MetricsUtil; @Slf4j @KinesisClientInternalApi public class ShardSyncTask implements ConsumerTask { - private final String SHARD_SYNC_TASK_OPERATION = "ShardSyncTask"; + private static final String SHARD_SYNC_TASK_OPERATION = "ShardSyncTask"; @NonNull private final ShardDetector shardDetector; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java index a6887f40..4bef8442 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java @@ -187,7 +187,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { public boolean createLeaseTableIfNotExists(@NonNull final Long readCapacity, @NonNull final Long writeCapacity) throws ProvisionedThroughputException, DependencyException { final CreateTableRequest.Builder builder = createTableRequestBuilder(); - if(BillingMode.PROVISIONED.equals(billingMode)) { + if (BillingMode.PROVISIONED.equals(billingMode)) { ProvisionedThroughput throughput = ProvisionedThroughput.builder().readCapacityUnits(readCapacity) .writeCapacityUnits(writeCapacity).build(); builder.provisionedThroughput(throughput); @@ -467,7 +467,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } catch (DynamoDbException | TimeoutException e) { throw convertAndRethrowExceptions("create", lease.leaseKey(), e); } - log.info("Created lease: {}",lease); + log.info("Created lease: {}", lease); return true; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java index 64a7840c..9ebed654 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java @@ -89,7 +89,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PENDING_CHECKPOINT_STATE_KEY, DynamoUtils.createAttributeValue(lease.checkpoint().subSequenceNumber())); } - if(lease.hashKeyRangeForLease() != null) { + if (lease.hashKeyRangeForLease() != null) { result.put(STARTING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey())); result.put(ENDING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey())); } @@ -274,7 +274,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); } - if(lease.hashKeyRangeForLease() != null) { + if (lease.hashKeyRangeForLease() != null) { result.put(STARTING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey()))); result.put(ENDING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey()))); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java index ba97ab08..8f2e8149 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java @@ -19,9 +19,9 @@ package software.amazon.kinesis.leases.exceptions; */ public class CustomerApplicationException extends Exception { - public CustomerApplicationException(Throwable e) { super(e);} + public CustomerApplicationException(Throwable e) { super(e); } - public CustomerApplicationException(String message, Throwable e) { super(message, e);} + public CustomerApplicationException(String message, Throwable e) { super(message, e); } - public CustomerApplicationException(String message) { super(message);} + public CustomerApplicationException(String message) { super(message); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java index c3f9523d..fb398cda 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java @@ -212,8 +212,10 @@ public class ProcessTask implements ConsumerTask { log.debug("Calling application processRecords() with {} records from {}", records.size(), shardInfoId); - final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records).cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime()) - .isAtShardEnd(input.isAtShardEnd()).checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build(); + final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records) + .cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime()) + .isAtShardEnd(input.isAtShardEnd()).checkpointer(recordProcessorCheckpointer) + .millisBehindLatest(input.millisBehindLatest()).build(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION); shardInfo.streamIdentifierSerOpt() diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java index e8406d92..7a034000 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java @@ -61,7 +61,7 @@ class ShardConsumerSubscriber implements Subscriber { @Deprecated ShardConsumerSubscriber(RecordsPublisher recordsPublisher, ExecutorService executorService, int bufferSize, ShardConsumer shardConsumer) { - this(recordsPublisher,executorService,bufferSize,shardConsumer, LifecycleConfig.DEFAULT_READ_TIMEOUTS_TO_IGNORE); + this(recordsPublisher, executorService, bufferSize, shardConsumer, LifecycleConfig.DEFAULT_READ_TIMEOUTS_TO_IGNORE); } ShardConsumerSubscriber(RecordsPublisher recordsPublisher, ExecutorService executorService, int bufferSize, @@ -74,7 +74,6 @@ class ShardConsumerSubscriber implements Subscriber { this.shardInfoId = ShardInfo.getLeaseKey(shardConsumer.shardInfo()); } - void startSubscriptions() { synchronized (lockObject) { // Setting the lastRequestTime to allow for health checks to restart subscriptions if they failed to diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 31bc8f88..4a96d87d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -283,7 +283,7 @@ public class ShutdownTask implements ConsumerTask { } } - for(ChildShard childShard : childShards) { + for (ChildShard childShard : childShards) { final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId()); if (leaseRefresher.getLease(leaseKey) == null) { log.debug("{} - Shard {} - Attempting to create lease for child shard {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseKey); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/CloudWatchMetricKey.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/CloudWatchMetricKey.java index 4b04cad7..e52893cd 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/CloudWatchMetricKey.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/CloudWatchMetricKey.java @@ -20,9 +20,7 @@ import java.util.Objects; import software.amazon.awssdk.services.cloudwatch.model.Dimension; import software.amazon.awssdk.services.cloudwatch.model.MetricDatum; - - -/* +/** * A representation of a key of a MetricDatum. This class is useful when wanting to compare * whether 2 keys have the same MetricDatum. This feature will be used in MetricAccumulatingQueue * where we aggregate metrics across multiple MetricScopes. @@ -48,12 +46,15 @@ public class CloudWatchMetricKey { @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } CloudWatchMetricKey other = (CloudWatchMetricKey) obj; return Objects.equals(other.dimensions, dimensions) && Objects.equals(other.metricName, metricName); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java index 5234ffe4..da83675f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java @@ -36,7 +36,6 @@ import java.util.Objects; * * MetricDatumWithKey sampleDatumWithKey = new MetricDatumWithKey(new * SampleMetricKey(System.currentTimeMillis()), datum) - * */ @AllArgsConstructor @Setter @@ -59,12 +58,15 @@ public class MetricDatumWithKey { @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } MetricDatumWithKey other = (MetricDatumWithKey) obj; return Objects.equals(other.key, key) && Objects.equals(other.datum, datum); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index 8404925d..3a8af8c2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java @@ -192,7 +192,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // Take action based on the time spent by the event in queue. takeDelayedDeliveryActionIfRequired(streamAndShardId, recordsRetrievedContext.getEnqueueTimestamp(), log); // Update current sequence number for the successfully delivered event. - currentSequenceNumber = ((FanoutRecordsRetrieved)recordsRetrieved).continuationSequenceNumber(); + currentSequenceNumber = ((FanoutRecordsRetrieved) recordsRetrieved).continuationSequenceNumber(); // Update the triggering flow for post scheduling upstream request. flowToBeReturned = recordsRetrievedContext.getRecordFlow(); // Try scheduling the next event in the queue or execute the subscription shutdown action. @@ -206,7 +206,8 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow != null && recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier() .equals(flow.getSubscribeToShardId())) { log.error( - "{}: Received unexpected ack for the active subscription {}. Throwing. ", streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); + "{}: Received unexpected ack for the active subscription {}. Throwing.", + streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); throw new IllegalStateException("Unexpected ack for the active subscription"); } // Otherwise publisher received a stale ack. @@ -315,7 +316,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { synchronized (lockObject) { if (!hasValidSubscriber()) { - if(hasValidFlow()) { + if (hasValidFlow()) { log.warn( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- Subscriber is null." + " Last successful request details -- {}", streamAndShardId, flow.connectionStartedAt, @@ -335,7 +336,8 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow != null) { String logMessage = String.format( "%s: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ %s id: %s -- %s." + - " Last successful request details -- %s", streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails); + " Last successful request details -- %s", streamAndShardId, flow.connectionStartedAt, + flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails); switch (category.throwableType) { case READ_TIMEOUT: log.debug(logMessage, propagationThrowable); @@ -778,7 +780,11 @@ public class FanOutRecordsPublisher implements RecordsPublisher { executeExceptionOccurred(throwable); } else { final SubscriptionShutdownEvent subscriptionShutdownEvent = new SubscriptionShutdownEvent( - () -> {parent.recordsDeliveryQueue.poll(); executeExceptionOccurred(throwable);}, "onError", throwable); + () -> { + parent.recordsDeliveryQueue.poll(); + executeExceptionOccurred(throwable); + }, + "onError", throwable); tryEnqueueSubscriptionShutdownEvent(subscriptionShutdownEvent); } } @@ -786,7 +792,6 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private void executeExceptionOccurred(Throwable throwable) { synchronized (parent.lockObject) { - log.debug("{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- {}: {}", parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); @@ -817,7 +822,11 @@ public class FanOutRecordsPublisher implements RecordsPublisher { executeComplete(); } else { final SubscriptionShutdownEvent subscriptionShutdownEvent = new SubscriptionShutdownEvent( - () -> {parent.recordsDeliveryQueue.poll(); executeComplete();}, "onComplete"); + () -> { + parent.recordsDeliveryQueue.poll(); + executeComplete(); + }, + "onComplete"); tryEnqueueSubscriptionShutdownEvent(subscriptionShutdownEvent); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java index 35301624..bcfb1081 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java @@ -54,7 +54,7 @@ public class FanOutRetrievalFactory implements RetrievalFactory { final StreamConfig streamConfig, final MetricsFactory metricsFactory) { final Optional streamIdentifierStr = shardInfo.streamIdentifierSerOpt(); - if(streamIdentifierStr.isPresent()) { + if (streamIdentifierStr.isPresent()) { final StreamIdentifier streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()); return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), getOrCreateConsumerArn(streamIdentifier, streamConfig.consumerArn()), diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java index 4dd64016..0cc7058d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PollingConfig.java @@ -137,7 +137,7 @@ public class PollingConfig implements RetrievalSpecificConfig { @Override public RetrievalFactory retrievalFactory() { // Prioritize the PollingConfig specified value if its updated. - if(usePollingConfigIdleTimeValue) { + if (usePollingConfigIdleTimeValue) { recordsFetcherFactory.idleMillisBetweenCalls(idleTimeBetweenReadsInMillis); } return new SynchronousBlockingRetrievalFactory(streamName(), kinesisClient(), recordsFetcherFactory, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index 07f4aaac..eb5937f7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -327,7 +327,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { } resetLock.writeLock().lock(); try { - publisherSession.reset((PrefetchRecordsRetrieved)recordsRetrieved); + publisherSession.reset((PrefetchRecordsRetrieved) recordsRetrieved); wasReset = true; } finally { resetLock.writeLock().unlock(); @@ -555,7 +555,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { return; } // Add a sleep if lastSuccessfulCall is still null but this is not the first try to avoid retry storm - if(lastSuccessfulCall == null) { + if (lastSuccessfulCall == null) { Thread.sleep(idleMillisBetweenCalls); return; } From eb6fd0cf32056e6df2a76d5064ca60848e2aa5f4 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Fri, 23 Jun 2023 16:15:33 -0400 Subject: [PATCH 70/88] Bound Checkstyle to `validate` goal for automated enforcement. (#1149) --- .../lifecycle/ShardConsumerSubscriber.java | 2 + .../fanout/FanOutRecordsPublisher.java | 14 +++++++ .../retrieval/polling/KinesisDataFetcher.java | 2 + checkstyle/checkstyle-suppressions.xml | 8 ++++ checkstyle/checkstyle.xml | 39 +++++++++++++++++++ pom.xml | 21 ++++++++++ 6 files changed, 86 insertions(+) create mode 100644 checkstyle/checkstyle-suppressions.xml create mode 100644 checkstyle/checkstyle.xml diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java index 7a034000..81528c4e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java @@ -130,7 +130,9 @@ class ShardConsumerSubscriber implements Subscriber { Duration timeSinceLastResponse = Duration.between(lastRequestTime, now); if (timeSinceLastResponse.toMillis() > maxTimeBetweenRequests) { log.error( + // CHECKSTYLE.OFF: LineLength "{}: Last request was dispatched at {}, but no response as of {} ({}). Cancelling subscription, and restarting. Last successful request details -- {}", + // CHECKSTYLE.ON: LineLength shardInfoId, lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails()); cancel(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index 3a8af8c2..ca4ce12d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java @@ -369,7 +369,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } else { if (triggeringFlow != null) { log.debug( + // CHECKSTYLE.OFF: LineLength "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- {} -> triggeringFlow wasn't the active flow. Didn't dispatch error", + // CHECKSTYLE.ON: LineLength streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId, category.throwableTypeString); triggeringFlow.cancel(); @@ -605,7 +607,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher { synchronized (lockObject) { if (subscriber != s) { log.warn( + // CHECKSTYLE.OFF: LineLength "{}: (FanOutRecordsPublisher/Subscription#request) - Rejected an attempt to request({}), because subscribers don't match. Last successful request details -- {}", + // CHECKSTYLE.ON: LineLength streamAndShardId, n, lastSuccessfulRequestDetails); return; } @@ -632,13 +636,17 @@ public class FanOutRecordsPublisher implements RecordsPublisher { synchronized (lockObject) { if (subscriber != s) { log.warn( + // CHECKSTYLE.OFF: LineLength "{}: (FanOutRecordsPublisher/Subscription#cancel) - Rejected attempt to cancel subscription, because subscribers don't match. Last successful request details -- {}", + // CHECKSTYLE.ON: LineLength streamAndShardId, lastSuccessfulRequestDetails); return; } if (!hasValidSubscriber()) { log.warn( + // CHECKSTYLE.OFF: LineLength "{}: (FanOutRecordsPublisher/Subscription#cancel) - Cancelled called even with an invalid subscriber. Last successful request details -- {}", + // CHECKSTYLE.ON: LineLength streamAndShardId, lastSuccessfulRequestDetails); } subscriber = null; @@ -808,7 +816,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher { isErrorDispatched = true; } else { log.debug( + // CHECKSTYLE.OFF: LineLength "{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- An error has previously been dispatched, not dispatching this error {}: {}", + // CHECKSTYLE.OFF: LineLength parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); } @@ -839,7 +849,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher { .add(new RecordsRetrievedContext(Either.right(subscriptionShutdownEvent), this, Instant.now())); } catch (Exception e) { log.warn( + // CHECKSTYLE.OFF: LineLength "{}: Unable to enqueue the {} shutdown event due to capacity restrictions in delivery queue with remaining capacity {}. Ignoring. Last successful request details -- {}", + // CHECKSTYLE.ON: LineLength parent.streamAndShardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(), parent.lastSuccessfulRequestDetails, subscriptionShutdownEvent.getShutdownEventThrowableOptional()); } @@ -863,7 +875,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } if (this.isDisposed) { log.warn( + // CHECKSTYLE.OFF: LineLength "{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- This flow has been disposed not dispatching completion. Last successful request details -- {}", + // CHECKSTYLE.ON: LineLength parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails); return; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java index 65da2b32..495dcfb1 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java @@ -145,7 +145,9 @@ public class KinesisDataFetcher implements DataFetcher { } } + // CHECKSTYLE.OFF: MemberName final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() { + // CHECKSTYLE.ON: MemberName @Override public GetRecordsResponse getResult() { return GetRecordsResponse.builder() diff --git a/checkstyle/checkstyle-suppressions.xml b/checkstyle/checkstyle-suppressions.xml new file mode 100644 index 00000000..91be49c8 --- /dev/null +++ b/checkstyle/checkstyle-suppressions.xml @@ -0,0 +1,8 @@ + + + + + + \ No newline at end of file diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml new file mode 100644 index 00000000..07932723 --- /dev/null +++ b/checkstyle/checkstyle.xml @@ -0,0 +1,39 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml index 7ebd6e54..82d7857d 100644 --- a/pom.xml +++ b/pom.xml @@ -72,6 +72,27 @@ + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.3.0 + + checkstyle/checkstyle.xml + true + true + checkstyle/checkstyle-suppressions.xml + + + + validate + + check + + + + + From dcd1c53fb10ac118406dbfe3a195a27b3a20922d Mon Sep 17 00:00:00 2001 From: pelaezryan Date: Mon, 26 Jun 2023 09:02:19 -0700 Subject: [PATCH 71/88] Update to Multilang Daemon to support StreamArn (#1143) * Updated multilang to support streamArn * Updated arn import to use software.amzon instead of com.amazonaws, also updated unit tests to be more explicit with the expected exceptions * Updated exception wording for region validation in StreamArn to be more consistent with other error messages * reverted spacing change * Updated StreamArn in multilang to only replace streamName (not region as well). Also updated unit tests and added Region validation * Updated region validation in multilang to be more readible * Refactored multilang unit tests to be more simple * Updated multilang daemon to validate streamArn based on pattern rather than individual section * removed region validation as this was not a requirement for stringArn support in multilangdaemon * removed spacing and removed unit test assertion on exception message * removed unnecessary param from unit test * removed unused imports from multilang unit tests * simplified the assertion for multilang daemon unit tests * Cleaned up unit test code following best practices for spacing/naming conventions and simplied kinesisClientLibConfiguration * Updated region code in unit tests for multilang daemon --------- Co-authored-by: Ryan Pelaez --- .../config/KinesisClientLibConfigurator.java | 21 ++- .../config/MultiLangDaemonConfiguration.java | 5 +- .../kinesis/multilang/MessageReaderTest.java | 1 - .../kinesis/multilang/MessageWriterTest.java | 3 - .../multilang/MultiLangDaemonConfigTest.java | 169 ++++++++++++++---- .../multilang/MultiLangDaemonTest.java | 1 - .../multilang/MultiLangProtocolTest.java | 3 - .../kinesis/multilang/ReadSTDERRTaskTest.java | 2 - ...eamingShardRecordProcessorFactoryTest.java | 3 - ...tialsProviderPropertyValueDecoderTest.java | 6 - .../KinesisClientLibConfiguratorTest.java | 41 +++-- .../multilang/messages/MessageTest.java | 7 - .../kinesis/common/StreamIdentifier.java | 14 +- 13 files changed, 195 insertions(+), 81 deletions(-) diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java index f3facdc0..c95d0853 100644 --- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java @@ -23,8 +23,10 @@ import org.apache.commons.beanutils.BeanUtilsBean; import org.apache.commons.beanutils.ConvertUtilsBean; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.regions.Region; +import software.amazon.kinesis.common.StreamIdentifier; /** * KinesisClientLibConfigurator constructs a KinesisClientLibConfiguration from java properties file. The following @@ -55,7 +57,7 @@ public class KinesisClientLibConfigurator { * Program will fail immediately, if customer provide: 1) invalid variable value. Program will log it as warning and * continue, if customer provide: 1) variable with unsupported variable type. 2) a variable with name which does not * match any of the variables in KinesisClientLibConfigration. - * + * * @param properties a Properties object containing the configuration information * @return KinesisClientLibConfiguration */ @@ -69,8 +71,19 @@ public class KinesisClientLibConfigurator { }); Validate.notBlank(configuration.getApplicationName(), "Application name is required"); - Validate.notBlank(configuration.getStreamName(), "Stream name is required"); + + if (configuration.getStreamArn() != null && !configuration.getStreamArn().trim().isEmpty()) { + final Arn streamArnObj = Arn.fromString(configuration.getStreamArn()); + StreamIdentifier.validateArn(streamArnObj); + //Parse out the stream Name from the Arn (and/or override existing value for Stream Name) + final String streamNameFromArn = streamArnObj.resource().resource(); + configuration.setStreamName(streamNameFromArn); + + } + + Validate.notBlank(configuration.getStreamName(), "Stream name or Stream Arn is required. Stream Arn takes precedence if both are passed in."); Validate.isTrue(configuration.getKinesisCredentialsProvider().isDirty(), "A basic set of AWS credentials must be provided"); + return configuration; } @@ -97,4 +110,4 @@ public class KinesisClientLibConfigurator { } -} +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java index da280ddf..7a7f2e79 100644 --- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java @@ -28,7 +28,6 @@ import java.util.UUID; import java.util.function.Function; import org.apache.commons.beanutils.BeanUtilsBean; -import org.apache.commons.beanutils.ConvertUtils; import org.apache.commons.beanutils.ConvertUtilsBean; import org.apache.commons.beanutils.Converter; import org.apache.commons.beanutils.converters.ArrayConverter; @@ -73,6 +72,8 @@ public class MultiLangDaemonConfiguration { private String applicationName; private String streamName; + private String streamArn; + @ConfigurationSettable(configurationClass = ConfigsBuilder.class) private String tableName; @@ -403,4 +404,4 @@ public class MultiLangDaemonConfiguration { return resolvedConfiguration(shardRecordProcessorFactory).build(); } -} +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java index b6541227..14ac357c 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java @@ -28,7 +28,6 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import software.amazon.kinesis.multilang.MessageReader; import software.amazon.kinesis.multilang.messages.Message; import software.amazon.kinesis.multilang.messages.StatusMessage; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java index 6a0c06b4..eaf6be7b 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java @@ -32,15 +32,12 @@ import org.mockito.Mockito; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; -import software.amazon.kinesis.multilang.MessageWriter; -import software.amazon.kinesis.multilang.messages.LeaseLostMessage; import software.amazon.kinesis.multilang.messages.Message; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; -import software.amazon.kinesis.lifecycle.ShutdownReason; import software.amazon.kinesis.retrieval.KinesisClientRecord; import static org.mockito.Mockito.verify; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java index b86a64ad..c6be1157 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java @@ -14,17 +14,14 @@ */ package software.amazon.kinesis.multilang; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.mockito.Matchers.any; import static org.mockito.Mockito.when; import java.io.ByteArrayInputStream; import java.io.IOException; -import java.util.Properties; -import org.apache.commons.beanutils.BeanUtilsBean; -import org.apache.commons.beanutils.ConvertUtilsBean; -import org.junit.Before; +import software.amazon.awssdk.regions.Region; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -39,48 +36,154 @@ import software.amazon.kinesis.multilang.config.MultiLangDaemonConfiguration; @RunWith(MockitoJUnitRunner.class) public class MultiLangDaemonConfigTest { - private static String FILENAME = "some.properties"; + private static final String FILENAME = "some.properties"; + private static final String EXE = "TestExe.exe"; + private static final String APPLICATION_NAME = MultiLangDaemonConfigTest.class.getSimpleName(); + private static final String STREAM_NAME = "fakeStream"; + private static final String STREAM_NAME_IN_ARN = "FAKE_STREAM_NAME"; + private static final Region REGION = Region.US_EAST_1; + private static final String STREAM_ARN = "arn:aws:kinesis:us-east-2:012345678987:stream/" + STREAM_NAME_IN_ARN; + + @Mock + private ClassLoader classLoader; @Mock private AwsCredentialsProvider credentialsProvider; @Mock private AwsCredentials creds; - @Mock + private KinesisClientLibConfigurator configurator; + private MultiLangDaemonConfig deamonConfig; - @Before - public void setup() { - ConvertUtilsBean convertUtilsBean = new ConvertUtilsBean(); - BeanUtilsBean utilsBean = new BeanUtilsBean(convertUtilsBean); - MultiLangDaemonConfiguration multiLangDaemonConfiguration = new MultiLangDaemonConfiguration(utilsBean, - convertUtilsBean); - multiLangDaemonConfiguration.setApplicationName("cool-app"); - multiLangDaemonConfiguration.setStreamName("cool-stream"); - multiLangDaemonConfiguration.setWorkerIdentifier("cool-worker"); - when(credentialsProvider.resolveCredentials()).thenReturn(creds); - when(creds.accessKeyId()).thenReturn("cool-user"); - when(configurator.getConfiguration(any(Properties.class))).thenReturn(multiLangDaemonConfiguration); - } + /** + * Instantiate a MultiLangDaemonConfig object + * @param streamName + * @param streamArn + * @throws IOException + */ + public void setup(String streamName, String streamArn) throws IOException { - @Test - public void constructorTest() throws IOException { - String PROPERTIES = "executableName = randomEXE \n" + "applicationName = testApp \n" - + "streamName = fakeStream \n" + "AWSCredentialsProvider = DefaultAWSCredentialsProviderChain\n" - + "processingLanguage = malbolge"; - ClassLoader classLoader = Mockito.mock(ClassLoader.class); + String properties = String.format("executableName = %s\n" + + "applicationName = %s\n" + + "AWSCredentialsProvider = DefaultAWSCredentialsProviderChain\n" + + "processingLanguage = malbolge\n" + + "regionName = %s\n", + EXE, + APPLICATION_NAME, + "us-east-1"); - Mockito.doReturn(new ByteArrayInputStream(PROPERTIES.getBytes())).when(classLoader) + if (streamName != null) { + properties += String.format("streamName = %s\n", streamName); + } + if (streamArn != null) { + properties += String.format("streamArn = %s\n", streamArn); + } + classLoader = Mockito.mock(ClassLoader.class); + + Mockito.doReturn(new ByteArrayInputStream(properties.getBytes())).when(classLoader) .getResourceAsStream(FILENAME); - MultiLangDaemonConfig deamonConfig = new MultiLangDaemonConfig(FILENAME, classLoader, configurator); + when(credentialsProvider.resolveCredentials()).thenReturn(creds); + when(creds.accessKeyId()).thenReturn("cool-user"); + configurator = new KinesisClientLibConfigurator(); - assertNotNull(deamonConfig.getExecutorService()); - assertNotNull(deamonConfig.getMultiLangDaemonConfiguration()); - assertNotNull(deamonConfig.getRecordProcessorFactory()); + deamonConfig = new MultiLangDaemonConfig(FILENAME, classLoader, configurator); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorFailsBecauseStreamArnIsInvalid() throws Exception { + setup("", "this_is_not_a_valid_arn"); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorFailsBecauseStreamArnIsInvalid2() throws Exception { + setup("", "arn:aws:kinesis:us-east-2:ACCOUNT_ID:BadFormatting:stream/" + STREAM_NAME_IN_ARN); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorFailsBecauseStreamNameAndArnAreEmpty() throws Exception { + setup("", ""); + } + + @Test(expected = NullPointerException.class) + public void testConstructorFailsBecauseStreamNameAndArnAreNull() throws Exception { + setup(null, null); + } + + @Test(expected = NullPointerException.class) + public void testConstructorFailsBecauseStreamNameIsNullAndArnIsEmpty() throws Exception { + setup(null, ""); + } + + @Test(expected = IllegalArgumentException.class) + public void testConstructorFailsBecauseStreamNameIsEmptyAndArnIsNull() throws Exception { + setup("", null); } @Test - public void propertyValidation() { + public void testConstructorUsingStreamName() throws IOException { + setup(STREAM_NAME, null); + + assertConfigurationsMatch(STREAM_NAME, null); + } + + @Test + public void testConstructorUsingStreamNameAndStreamArnIsEmpty() throws IOException { + setup(STREAM_NAME, ""); + + assertConfigurationsMatch(STREAM_NAME, ""); + } + + @Test + public void testConstructorUsingStreamNameAndStreamArnIsWhitespace() throws IOException { + setup(STREAM_NAME, " "); + + assertConfigurationsMatch(STREAM_NAME, ""); + } + + @Test + public void testConstructorUsingStreamArn() throws IOException { + setup(null, STREAM_ARN); + + assertConfigurationsMatch(STREAM_NAME_IN_ARN, STREAM_ARN); + } + + @Test + public void testConstructorUsingStreamNameAsEmptyAndStreamArn() throws IOException { + setup("", STREAM_ARN); + + assertConfigurationsMatch(STREAM_NAME_IN_ARN, STREAM_ARN); + } + + @Test + public void testConstructorUsingStreamArnOverStreamName() throws IOException { + setup(STREAM_NAME, STREAM_ARN); + + assertConfigurationsMatch(STREAM_NAME_IN_ARN, STREAM_ARN); + } + + /** + * Verify the daemonConfig properties are what we expect them to be. + * @param deamonConfig + * @param expectedStreamName + */ + private void assertConfigurationsMatch(String expectedStreamName, String expectedStreamArn) { + final MultiLangDaemonConfiguration multiLangConfiguration = deamonConfig.getMultiLangDaemonConfiguration(); + assertNotNull(deamonConfig.getExecutorService()); + assertNotNull(multiLangConfiguration); + assertNotNull(deamonConfig.getRecordProcessorFactory()); + + assertEquals(EXE, deamonConfig.getRecordProcessorFactory().getCommandArray()[0]); + assertEquals(APPLICATION_NAME, multiLangConfiguration.getApplicationName()); + assertEquals(expectedStreamName, multiLangConfiguration.getStreamName()); + assertEquals(REGION, multiLangConfiguration.getDynamoDbClient().get("region")); + assertEquals(REGION, multiLangConfiguration.getCloudWatchClient().get("region")); + assertEquals(REGION, multiLangConfiguration.getKinesisClient().get("region")); + assertEquals(expectedStreamArn, multiLangConfiguration.getStreamArn()); + } + + @Test + public void testPropertyValidation() { String PROPERTIES_NO_EXECUTABLE_NAME = "applicationName = testApp \n" + "streamName = fakeStream \n" + "AWSCredentialsProvider = DefaultAWSCredentialsProviderChain\n" + "processingLanguage = malbolge"; ClassLoader classLoader = Mockito.mock(ClassLoader.class); @@ -99,4 +202,4 @@ public class MultiLangDaemonConfigTest { } } -} +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonTest.java index 0c1d0b60..3229e2b8 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonTest.java @@ -17,7 +17,6 @@ package software.amazon.kinesis.multilang; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.isEmptyOrNullString; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; import static org.mockito.Matchers.anyObject; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java index dc6166aa..d385b2f9 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java @@ -31,7 +31,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -61,10 +60,8 @@ import software.amazon.kinesis.multilang.messages.ShardEndedMessage; import software.amazon.kinesis.multilang.messages.StatusMessage; import com.google.common.util.concurrent.SettableFuture; -import software.amazon.kinesis.coordinator.KinesisClientLibConfiguration; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; -import software.amazon.kinesis.lifecycle.ShutdownReason; import software.amazon.kinesis.processor.RecordProcessorCheckpointer; import software.amazon.kinesis.retrieval.KinesisClientRecord; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java index b3bb0719..bffd431d 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java @@ -27,8 +27,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import software.amazon.kinesis.multilang.DrainChildSTDERRTask; -import software.amazon.kinesis.multilang.LineReaderTask; public class ReadSTDERRTaskTest { diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorFactoryTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorFactoryTest.java index 7a7d7b11..1954cf91 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorFactoryTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorFactoryTest.java @@ -14,12 +14,9 @@ */ package software.amazon.kinesis.multilang; -import software.amazon.kinesis.coordinator.KinesisClientLibConfiguration; import org.junit.Assert; import org.junit.Test; -import software.amazon.kinesis.multilang.MultiLangRecordProcessorFactory; -import software.amazon.kinesis.multilang.MultiLangShardRecordProcessor; import software.amazon.kinesis.multilang.config.MultiLangDaemonConfiguration; import software.amazon.kinesis.processor.ShardRecordProcessor; import org.junit.runner.RunWith; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java index 8da22d53..36f496d3 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java @@ -16,7 +16,6 @@ package software.amazon.kinesis.multilang.config; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import java.util.Arrays; @@ -32,11 +31,6 @@ import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSCredentialsProviderChain; -import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; -import software.amazon.awssdk.auth.credentials.AwsCredentials; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain; - public class AWSCredentialsProviderPropertyValueDecoderTest { private static final String TEST_ACCESS_KEY_ID = "123"; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java index 031fc427..1f05240a 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java @@ -278,10 +278,8 @@ public class KinesisClientLibConfiguratorTest { } } - @Test + @Test(expected = IllegalArgumentException.class) public void testWithMissingCredentialsProvider() { - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("A basic set of AWS credentials must be provided"); String test = StringUtils.join(new String[] { "streamName = a", "applicationName = b", "workerId = 123", "failoverTimeMillis = 100", "shardSyncIntervalMillis = 500" }, '\n'); @@ -305,22 +303,37 @@ public class KinesisClientLibConfiguratorTest { assertFalse(config.getWorkerIdentifier().isEmpty()); } - @Test - public void testWithMissingStreamName() { - thrown.expect(NullPointerException.class); - thrown.expectMessage("Stream name is required"); - - String test = StringUtils.join(new String[] { "applicationName = b", - "AWSCredentialsProvider = " + credentialName1, "workerId = 123", "failoverTimeMillis = 100" }, '\n'); + @Test(expected = NullPointerException.class) + public void testWithMissingStreamNameAndMissingStreamArn() { + String test = StringUtils.join(new String[] { + "applicationName = b", + "AWSCredentialsProvider = " + credentialName1, + "workerId = 123", + "failoverTimeMillis = 100" }, + '\n'); InputStream input = new ByteArrayInputStream(test.getBytes()); configurator.getConfiguration(input); } - @Test + @Test(expected = IllegalArgumentException.class) + public void testWithEmptyStreamNameAndMissingStreamArn() { + + String test = StringUtils.join(new String[] { + "applicationName = b", + "AWSCredentialsProvider = " + credentialName1, + "workerId = 123", + "failoverTimeMillis = 100", + "streamName = ", + "streamArn = "}, + '\n'); + InputStream input = new ByteArrayInputStream(test.getBytes()); + + configurator.getConfiguration(input); + } + + @Test(expected = NullPointerException.class) public void testWithMissingApplicationName() { - thrown.expect(NullPointerException.class); - thrown.expectMessage("Application name is required"); String test = StringUtils.join(new String[] { "streamName = a", "AWSCredentialsProvider = " + credentialName1, "workerId = 123", "failoverTimeMillis = 100" }, '\n'); @@ -493,4 +506,4 @@ public class KinesisClientLibConfiguratorTest { MultiLangDaemonConfiguration config = configurator.getConfiguration(input); return config; } -} +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java index 47337221..86798080 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java @@ -26,13 +26,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.lifecycle.ShutdownReason; -import software.amazon.kinesis.multilang.messages.CheckpointMessage; -import software.amazon.kinesis.multilang.messages.InitializeMessage; -import software.amazon.kinesis.multilang.messages.Message; -import software.amazon.kinesis.multilang.messages.ProcessRecordsMessage; -import software.amazon.kinesis.multilang.messages.ShutdownMessage; -import software.amazon.kinesis.multilang.messages.ShutdownRequestedMessage; -import software.amazon.kinesis.multilang.messages.StatusMessage; import software.amazon.kinesis.retrieval.KinesisClientRecord; public class MessageTest { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java index 8307ed82..82cef04b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -167,12 +167,22 @@ public class StreamIdentifier { .build(); } - private static void validateArn(Arn streamArn) { + /** + * Verify the streamArn follows the appropriate formatting. + * Throw an exception if it does not. + * @param streamArn + */ + public static void validateArn(Arn streamArn) { if (!STREAM_ARN_PATTERN.matcher(streamArn.toString()).matches() || !streamArn.region().isPresent()) { - throw new IllegalArgumentException("Unable to create a StreamIdentifier from " + streamArn); + throw new IllegalArgumentException("Invalid streamArn " + streamArn); } } + /** + * Verify creationEpoch is greater than 0. + * Throw an exception if it is not. + * @param creationEpoch + */ private static void validateCreationEpoch(long creationEpoch) { if (creationEpoch <= 0) { throw new IllegalArgumentException( From 5105317eb4b8a012feec32209920b6b31e364d0f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Jun 2023 15:11:34 -0400 Subject: [PATCH 72/88] Bump guava from 31.1-jre to 32.0.0-jre in /amazon-kinesis-client (#1142) Bumps [guava](https://github.com/google/guava) from 31.1-jre to 32.0.0-jre. - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) --- updated-dependencies: - dependency-name: com.google.guava:guava dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- amazon-kinesis-client/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 3eb8ad94..30824281 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -88,7 +88,7 @@ com.google.guava guava - 31.1-jre + 32.0.0-jre com.google.protobuf From 74d8f4b780a53af09830acc957248902ede44248 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 26 Jun 2023 15:25:10 -0400 Subject: [PATCH 73/88] Enabled Checkstyle validation of test resources. (#1150) No functional change. --- .../kinesis/multilang/MessageReaderTest.java | 21 ++++---- .../kinesis/multilang/MessageWriterTest.java | 14 +++--- .../multilang/MultiLangDaemonConfigTest.java | 9 ++-- .../kinesis/multilang/ReadSTDERRTaskTest.java | 8 +-- .../StreamingShardRecordProcessorTest.java | 50 ++++++++----------- .../kinesis/checkpoint/CheckpointerTest.java | 10 ++-- .../checkpoint/InMemoryCheckpointer.java | 14 +++--- ...dShardRecordProcessorCheckpointerTest.java | 34 ++++++------- ...sticShuffleShardSyncLeaderDeciderTest.java | 2 +- .../coordinator/DiagnosticEventsTest.java | 8 +-- .../PeriodicShardSyncManagerTest.java | 14 +++--- .../kinesis/coordinator/SchedulerTest.java | 22 ++++---- .../kinesis/coordinator/WorkerTest.java | 21 ++++---- .../leases/HierarchicalShardSyncerTest.java | 22 +++++--- .../leases/LeaseCoordinatorExerciser.java | 15 +++--- .../kinesis/leases/ShardObjectHelper.java | 6 +-- ...llingModePayPerRequestIntegrationTest.java | 2 +- .../DynamoDBLeaseRenewerIntegrationTest.java | 2 +- .../dynamodb/DynamoDBLeaseRenewerTest.java | 2 +- .../metrics/MetricAccumulatingQueueTest.java | 4 +- .../fanout/FanOutRecordsPublisherTest.java | 42 +++++++++------- .../polling/KinesisDataFetcherTest.java | 2 +- ...efetchRecordsPublisherIntegrationTest.java | 4 +- .../polling/PrefetchRecordsPublisherTest.java | 16 +++--- .../amazon/kinesis/utils/BlockingUtils.java | 6 +-- .../utils/SubscribeToShardRequestMatcher.java | 2 +- checkstyle/checkstyle.xml | 5 +- pom.xml | 1 + 28 files changed, 183 insertions(+), 175 deletions(-) diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java index 14ac357c..f6fab4c1 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageReaderTest.java @@ -34,9 +34,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; public class MessageReaderTest { - private static final String shardId = "shard-123"; + private static final String SHARD_ID = "shard-123"; - /* + /** * This line is based on the definition of the protocol for communication between the KCL record processor and * the client's process. */ @@ -44,7 +44,7 @@ public class MessageReaderTest { return String.format("{\"action\":\"checkpoint\", \"checkpoint\":\"%s\"}", sequenceNumber); } - /* + /** * This line is based on the definition of the protocol for communication between the KCL record processor and * the client's process. */ @@ -79,10 +79,9 @@ public class MessageReaderTest { String[] responseFors = new String[] { "initialize", "processRecords", "processRecords", "shutdown" }; InputStream stream = buildInputStreamOfGoodInput(sequenceNumbers, responseFors); MessageReader reader = - new MessageReader().initialize(stream, shardId, new ObjectMapper(), Executors.newCachedThreadPool()); + new MessageReader().initialize(stream, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); for (String responseFor : responseFors) { - StatusMessage statusMessage = null; try { Message message = reader.getNextMessageFromSTDOUT().get(); if (message instanceof StatusMessage) { @@ -102,14 +101,14 @@ public class MessageReaderTest { InputStream stream = buildInputStreamOfGoodInput(sequenceNumbers, responseFors); MessageReader reader = - new MessageReader().initialize(stream, shardId, new ObjectMapper(), Executors.newCachedThreadPool()); + new MessageReader().initialize(stream, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); Future drainFuture = reader.drainSTDOUT(); Boolean drainResult = drainFuture.get(); Assert.assertNotNull(drainResult); Assert.assertTrue(drainResult); } - /* + /** * readValue should fail safely and just continue looping */ @Test @@ -134,7 +133,7 @@ public class MessageReaderTest { } MessageReader reader = - new MessageReader().initialize(bufferReader, shardId, new ObjectMapper(), + new MessageReader().initialize(bufferReader, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); try { @@ -149,7 +148,7 @@ public class MessageReaderTest { public void messageReaderBuilderTest() { InputStream stream = new ByteArrayInputStream("".getBytes()); MessageReader reader = - new MessageReader().initialize(stream, shardId, new ObjectMapper(), Executors.newCachedThreadPool()); + new MessageReader().initialize(stream, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); Assert.assertNotNull(reader); } @@ -158,7 +157,7 @@ public class MessageReaderTest { BufferedReader input = Mockito.mock(BufferedReader.class); Mockito.doThrow(IOException.class).when(input).readLine(); MessageReader reader = - new MessageReader().initialize(input, shardId, new ObjectMapper(), Executors.newCachedThreadPool()); + new MessageReader().initialize(input, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); Future readTask = reader.getNextMessageFromSTDOUT(); @@ -176,7 +175,7 @@ public class MessageReaderTest { public void noMoreMessagesTest() throws InterruptedException { InputStream stream = new ByteArrayInputStream("".getBytes()); MessageReader reader = - new MessageReader().initialize(stream, shardId, new ObjectMapper(), Executors.newCachedThreadPool()); + new MessageReader().initialize(stream, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); Future future = reader.getNextMessageFromSTDOUT(); try { diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java index eaf6be7b..c997c193 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java @@ -44,7 +44,7 @@ import static org.mockito.Mockito.verify; public class MessageWriterTest { - private static final String shardId = "shard-123"; + private static final String SHARD_ID = "shard-123"; MessageWriter messageWriter; OutputStream stream; @@ -57,7 +57,7 @@ public class MessageWriterTest { public void setup() { stream = Mockito.mock(OutputStream.class); messageWriter = - new MessageWriter().initialize(stream, shardId, new ObjectMapper(), Executors.newCachedThreadPool()); + new MessageWriter().initialize(stream, SHARD_ID, new ObjectMapper(), Executors.newCachedThreadPool()); } /* @@ -83,7 +83,7 @@ public class MessageWriterTest { @Test public void writeInitializeMessageTest() throws IOException, InterruptedException, ExecutionException { - Future future = this.messageWriter.writeInitializeMessage(InitializationInput.builder().shardId(shardId).build()); + Future future = this.messageWriter.writeInitializeMessage(InitializationInput.builder().shardId(SHARD_ID).build()); future.get(); verify(this.stream, Mockito.atLeastOnce()).write(Mockito.any(byte[].class), Mockito.anyInt(), Mockito.anyInt()); @@ -128,20 +128,20 @@ public class MessageWriterTest { @Test public void streamIOExceptionTest() throws IOException, InterruptedException, ExecutionException { Mockito.doThrow(IOException.class).when(stream).flush(); - Future initializeTask = this.messageWriter.writeInitializeMessage(InitializationInput.builder().shardId(shardId).build()); + Future initializeTask = this.messageWriter.writeInitializeMessage(InitializationInput.builder().shardId(SHARD_ID).build()); Boolean result = initializeTask.get(); Assert.assertNotNull(result); Assert.assertFalse(result); } @Test - public void objectMapperFails() throws JsonProcessingException, InterruptedException, ExecutionException { + public void objectMapperFails() throws JsonProcessingException { thrown.expect(RuntimeException.class); thrown.expectMessage("Encountered I/O error while writing LeaseLostMessage action to subprocess"); ObjectMapper mapper = Mockito.mock(ObjectMapper.class); Mockito.doThrow(JsonProcessingException.class).when(mapper).writeValueAsString(Mockito.any(Message.class)); - messageWriter = new MessageWriter().initialize(stream, shardId, mapper, Executors.newCachedThreadPool()); + messageWriter = new MessageWriter().initialize(stream, SHARD_ID, mapper, Executors.newCachedThreadPool()); messageWriter.writeLeaseLossMessage(LeaseLostInput.builder().build()); } @@ -154,7 +154,7 @@ public class MessageWriterTest { Assert.assertFalse(this.messageWriter.isOpen()); try { // Any message should fail - this.messageWriter.writeInitializeMessage(InitializationInput.builder().shardId(shardId).build()); + this.messageWriter.writeInitializeMessage(InitializationInput.builder().shardId(SHARD_ID).build()); Assert.fail("MessageWriter should be closed and unable to write."); } catch (IllegalStateException e) { // This should happen. diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java index c6be1157..c5740a2f 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java @@ -164,7 +164,7 @@ public class MultiLangDaemonConfigTest { /** * Verify the daemonConfig properties are what we expect them to be. - * @param deamonConfig + * * @param expectedStreamName */ private void assertConfigurationsMatch(String expectedStreamName, String expectedStreamArn) { @@ -184,16 +184,15 @@ public class MultiLangDaemonConfigTest { @Test public void testPropertyValidation() { - String PROPERTIES_NO_EXECUTABLE_NAME = "applicationName = testApp \n" + "streamName = fakeStream \n" + String propertiesNoExecutableName = "applicationName = testApp \n" + "streamName = fakeStream \n" + "AWSCredentialsProvider = DefaultAWSCredentialsProviderChain\n" + "processingLanguage = malbolge"; ClassLoader classLoader = Mockito.mock(ClassLoader.class); - Mockito.doReturn(new ByteArrayInputStream(PROPERTIES_NO_EXECUTABLE_NAME.getBytes())).when(classLoader) + Mockito.doReturn(new ByteArrayInputStream(propertiesNoExecutableName.getBytes())).when(classLoader) .getResourceAsStream(FILENAME); - MultiLangDaemonConfig config; try { - config = new MultiLangDaemonConfig(FILENAME, classLoader, configurator); + new MultiLangDaemonConfig(FILENAME, classLoader, configurator); Assert.fail("Construction of the config should have failed due to property validation failing."); } catch (IllegalArgumentException e) { // Good diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java index bffd431d..45ff3052 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/ReadSTDERRTaskTest.java @@ -30,7 +30,7 @@ import org.mockito.Mockito; public class ReadSTDERRTaskTest { - private static final String shardId = "shard-123"; + private static final String SHARD_ID = "shard-123"; private BufferedReader mockBufferReader; @Before @@ -43,7 +43,7 @@ public class ReadSTDERRTaskTest { String errorMessages = "OMG\nThis is test message\n blah blah blah \n"; InputStream stream = new ByteArrayInputStream(errorMessages.getBytes()); - LineReaderTask reader = new DrainChildSTDERRTask().initialize(stream, shardId, ""); + LineReaderTask reader = new DrainChildSTDERRTask().initialize(stream, SHARD_ID, ""); Assert.assertNotNull(reader); } @@ -52,7 +52,7 @@ public class ReadSTDERRTaskTest { String errorMessages = "OMG\nThis is test message\n blah blah blah \n"; BufferedReader bufferReader = new BufferedReader(new InputStreamReader(new ByteArrayInputStream(errorMessages.getBytes()))); - LineReaderTask errorReader = new DrainChildSTDERRTask().initialize(bufferReader, shardId, ""); + LineReaderTask errorReader = new DrainChildSTDERRTask().initialize(bufferReader, SHARD_ID, ""); Assert.assertNotNull(errorReader); Boolean result = errorReader.call(); @@ -65,7 +65,7 @@ public class ReadSTDERRTaskTest { } catch (IOException e) { Assert.fail("Not supposed to get an exception when we're just building our mock."); } - LineReaderTask errorReader = new DrainChildSTDERRTask().initialize(mockBufferReader, shardId, ""); + LineReaderTask errorReader = new DrainChildSTDERRTask().initialize(mockBufferReader, SHARD_ID, ""); Assert.assertNotNull(errorReader); Future result = Executors.newCachedThreadPool().submit(errorReader); Boolean finishedCleanly = null; diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java index e3368e07..caa925b0 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java @@ -23,7 +23,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Collections; @@ -46,9 +45,7 @@ import org.mockito.stubbing.Answer; import com.fasterxml.jackson.databind.ObjectMapper; import software.amazon.awssdk.services.kinesis.model.Record; -import software.amazon.kinesis.exceptions.InvalidStateException; import software.amazon.kinesis.exceptions.KinesisClientLibDependencyException; -import software.amazon.kinesis.exceptions.ShutdownException; import software.amazon.kinesis.exceptions.ThrottlingException; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; @@ -67,7 +64,7 @@ import software.amazon.kinesis.retrieval.KinesisClientRecord; @RunWith(MockitoJUnitRunner.class) public class StreamingShardRecordProcessorTest { - private static final String shardId = "shard-123"; + private static final String SHARD_ID = "shard-123"; private int systemExitCount = 0; @@ -79,77 +76,73 @@ public class StreamingShardRecordProcessorTest { private RecordProcessorCheckpointer unimplementedCheckpointer = new RecordProcessorCheckpointer() { @Override - public void checkpoint() throws KinesisClientLibDependencyException, InvalidStateException, - ThrottlingException, ShutdownException { + public void checkpoint() throws KinesisClientLibDependencyException, ThrottlingException { throw new UnsupportedOperationException(); } @Override public void checkpoint(String sequenceNumber) throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + ThrottlingException, IllegalArgumentException { throw new UnsupportedOperationException(); } @Override public void checkpoint(Record record) - throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException { + throws KinesisClientLibDependencyException, ThrottlingException { throw new UnsupportedOperationException(); } @Override public void checkpoint(String sequenceNumber, long subSequenceNumber) - throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException, - IllegalArgumentException { + throws KinesisClientLibDependencyException, ThrottlingException, IllegalArgumentException { throw new UnsupportedOperationException(); } @Override public PreparedCheckpointer prepareCheckpoint() - throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException { + throws KinesisClientLibDependencyException, ThrottlingException { throw new UnsupportedOperationException(); } @Override - public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) + throws KinesisClientLibDependencyException, ThrottlingException { throw new UnsupportedOperationException(); } @Override public PreparedCheckpointer prepareCheckpoint(Record record) - throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException { + throws KinesisClientLibDependencyException, ThrottlingException { throw new UnsupportedOperationException(); } @Override - public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) + throws KinesisClientLibDependencyException, ThrottlingException { throw new UnsupportedOperationException(); } @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber) - throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + throws KinesisClientLibDependencyException, ThrottlingException, IllegalArgumentException { throw new UnsupportedOperationException(); } @Override - public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, ThrottlingException, IllegalArgumentException { return null; } @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) - throws KinesisClientLibDependencyException, - InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + throws KinesisClientLibDependencyException, ThrottlingException, IllegalArgumentException { throw new UnsupportedOperationException(); } @Override - public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, ThrottlingException, IllegalArgumentException { throw new UnsupportedOperationException(); } @@ -171,7 +164,7 @@ public class StreamingShardRecordProcessorTest { private MultiLangDaemonConfiguration configuration; @Before - public void prepare() throws IOException, InterruptedException, ExecutionException { + public void prepare() throws InterruptedException, ExecutionException { // Fake command systemExitCount = 0; @@ -230,7 +223,7 @@ public class StreamingShardRecordProcessorTest { List testRecords = Collections.emptyList(); - recordProcessor.initialize(InitializationInput.builder().shardId(shardId).build()); + recordProcessor.initialize(InitializationInput.builder().shardId(SHARD_ID).build()); recordProcessor.processRecords(ProcessRecordsInput.builder().records(testRecords) .checkpointer(unimplementedCheckpointer).build()); recordProcessor.processRecords(ProcessRecordsInput.builder().records(testRecords) @@ -240,7 +233,6 @@ public class StreamingShardRecordProcessorTest { @Test public void processorPhasesTest() throws InterruptedException, ExecutionException { - Answer answer = new Answer() { StatusMessage[] answers = new StatusMessage[] { new StatusMessage(InitializeMessage.ACTION), @@ -263,7 +255,7 @@ public class StreamingShardRecordProcessorTest { verify(messageWriter) .writeInitializeMessage(argThat(Matchers.withInit( - InitializationInput.builder().shardId(shardId).build()))); + InitializationInput.builder().shardId(SHARD_ID).build()))); verify(messageWriter, times(2)).writeProcessRecordsMessage(any(ProcessRecordsInput.class)); verify(messageWriter).writeLeaseLossMessage(any(LeaseLostInput.class)); } @@ -295,7 +287,7 @@ public class StreamingShardRecordProcessorTest { phases(answer); verify(messageWriter).writeInitializeMessage(argThat(Matchers.withInit(InitializationInput.builder() - .shardId(shardId).build()))); + .shardId(SHARD_ID).build()))); verify(messageWriter, times(2)).writeProcessRecordsMessage(any(ProcessRecordsInput.class)); verify(messageWriter, never()).writeLeaseLossMessage(any(LeaseLostInput.class)); Assert.assertEquals(1, systemExitCount); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java index b823c8e3..eb341238 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java @@ -37,9 +37,9 @@ public class CheckpointerTest { @Test public final void testInitialSetCheckpoint() throws Exception { - String sequenceNumber = "1"; + String sequenceNumber = "1"; String shardId = "myShardId"; - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber); checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken); ExtendedSequenceNumber registeredCheckpoint = checkpoint.getCheckpoint(shardId); Assert.assertEquals(extendedSequenceNumber, registeredCheckpoint); @@ -49,8 +49,8 @@ public class CheckpointerTest { public final void testAdvancingSetCheckpoint() throws Exception { String shardId = "myShardId"; for (Integer i = 0; i < 10; i++) { - String sequenceNumber = i.toString(); - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber); + String sequenceNumber = i.toString(); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber); checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken); ExtendedSequenceNumber registeredCheckpoint = checkpoint.getCheckpoint(shardId); Assert.assertEquals(extendedSequenceNumber, registeredCheckpoint); @@ -67,7 +67,7 @@ public class CheckpointerTest { String checkpointValue = "12345"; String shardId = "testShardId-1"; String concurrencyToken = "token-1"; - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(checkpointValue); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(checkpointValue); checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), concurrencyToken); Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId)); Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java index 8f6e165d..a2d83568 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java @@ -39,8 +39,7 @@ public class InMemoryCheckpointer implements Checkpointer { * {@inheritDoc} */ @Override - public void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken) - throws KinesisClientLibException { + public void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken) { checkpoints.put(leaseKey, checkpointValue); flushpoints.put(leaseKey, checkpointValue); pendingCheckpoints.remove(leaseKey); @@ -49,33 +48,32 @@ public class InMemoryCheckpointer implements Checkpointer { if (log.isDebugEnabled()) { log.debug("shardId: {} checkpoint: {}", leaseKey, checkpointValue); } - } /** * {@inheritDoc} */ @Override - public ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException { + public ExtendedSequenceNumber getCheckpoint(String leaseKey) { ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); log.debug("checkpoint shardId: {} checkpoint: {}", leaseKey, checkpoint); return checkpoint; } @Override - public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) - throws KinesisClientLibException { + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) { prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null); } @Override - public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException { + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, + byte[] pendingCheckpointState) { pendingCheckpoints.put(leaseKey, pendingCheckpoint); pendingCheckpointStates.put(leaseKey, pendingCheckpointState); } @Override - public Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException { + public Checkpoint getCheckpointObject(String leaseKey) { ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(leaseKey); byte[] pendingCheckpointState = pendingCheckpointStates.get(leaseKey); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java index 2ff82004..37a40b6b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java @@ -91,11 +91,11 @@ public class ShardShardRecordProcessorCheckpointerTest { */ @Test public final void testCheckpointRecord() throws Exception { - ShardRecordProcessorCheckpointer processingCheckpointer = + ShardRecordProcessorCheckpointer processingCheckpointer = new ShardRecordProcessorCheckpointer(shardInfo, checkpoint); - processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5025"); - Record record = makeRecord("5025"); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5025"); + Record record = makeRecord("5025"); processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); processingCheckpointer.checkpoint(record); assertThat(checkpoint.getCheckpoint(shardId), equalTo(extendedSequenceNumber)); @@ -107,13 +107,13 @@ public class ShardShardRecordProcessorCheckpointerTest { */ @Test public final void testCheckpointSubRecord() throws Exception { - ShardRecordProcessorCheckpointer processingCheckpointer = + ShardRecordProcessorCheckpointer processingCheckpointer = new ShardRecordProcessorCheckpointer(shardInfo, checkpoint); - processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5030"); - Record record = makeRecord("5030"); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5030"); + Record record = makeRecord("5030"); //UserRecord subRecord = new UserRecord(record); - processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); + processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); processingCheckpointer.checkpoint(record); assertThat(checkpoint.getCheckpoint(shardId), equalTo(extendedSequenceNumber)); } @@ -124,11 +124,11 @@ public class ShardShardRecordProcessorCheckpointerTest { */ @Test public final void testCheckpointSequenceNumber() throws Exception { - ShardRecordProcessorCheckpointer processingCheckpointer = + ShardRecordProcessorCheckpointer processingCheckpointer = new ShardRecordProcessorCheckpointer(shardInfo, checkpoint); - processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5035"); - processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5035"); + processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); processingCheckpointer.checkpoint("5035"); assertThat(checkpoint.getCheckpoint(shardId), equalTo(extendedSequenceNumber)); } @@ -139,11 +139,11 @@ public class ShardShardRecordProcessorCheckpointerTest { */ @Test public final void testCheckpointExtendedSequenceNumber() throws Exception { - ShardRecordProcessorCheckpointer processingCheckpointer = + ShardRecordProcessorCheckpointer processingCheckpointer = new ShardRecordProcessorCheckpointer(shardInfo, checkpoint); - processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); - ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5040"); - processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5040"); + processingCheckpointer.largestPermittedCheckpointValue(extendedSequenceNumber); processingCheckpointer.checkpoint("5040", 0); assertThat(checkpoint.getCheckpoint(shardId), equalTo(extendedSequenceNumber)); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java index dff2a8cb..9508903b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java @@ -105,7 +105,7 @@ public class DeterministicShuffleShardSyncLeaderDeciderTest { @Test public void testElectedLeadersAsPerExpectedShufflingOrder() throws Exception { - List leases = getLeases(5, false /*emptyLeaseOwner */,false /* duplicateLeaseOwner */, true /* activeLeases */); + List leases = getLeases(5, false /*emptyLeaseOwner */, false /* duplicateLeaseOwner */, true /* activeLeases */); when(leaseRefresher.listLeases()).thenReturn(leases); Set expectedLeaders = getExpectedLeaders(leases); for (String leader : expectedLeaders) { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DiagnosticEventsTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DiagnosticEventsTest.java index d6098cca..08ed8abb 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DiagnosticEventsTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DiagnosticEventsTest.java @@ -86,7 +86,7 @@ public class DiagnosticEventsTest { assertEquals(event.getLargestPoolSize(), largestPoolSize); assertEquals(event.getMaximumPoolSize(), maximumPoolSize); assertEquals(event.getLeasesOwned(), leaseAssignments.size()); - assertEquals(event.getCurrentQueueSize(),0); + assertEquals(0, event.getCurrentQueueSize()); verify(defaultHandler, times(1)).visit(event); } @@ -110,7 +110,7 @@ public class DiagnosticEventsTest { assertEquals(event.getExecutorStateEvent().getLargestPoolSize(), largestPoolSize); assertEquals(event.getExecutorStateEvent().getMaximumPoolSize(), maximumPoolSize); assertEquals(event.getExecutorStateEvent().getLeasesOwned(), leaseAssignments.size()); - assertEquals(event.getExecutorStateEvent().getCurrentQueueSize(),0); + assertEquals(0, event.getExecutorStateEvent().getCurrentQueueSize()); assertTrue(event.getThrowable() instanceof TestRejectedTaskException); verify(defaultHandler, times(1)).visit(event); @@ -136,7 +136,7 @@ public class DiagnosticEventsTest { assertEquals(executorStateEvent.getLargestPoolSize(), largestPoolSize); assertEquals(executorStateEvent.getMaximumPoolSize(), maximumPoolSize); assertEquals(executorStateEvent.getLeasesOwned(), leaseAssignments.size()); - assertEquals(executorStateEvent.getCurrentQueueSize(),0); + assertEquals(0, executorStateEvent.getCurrentQueueSize()); RejectedTaskEvent rejectedTaskEvent = factory.rejectedTaskEvent(executorStateEvent, new TestRejectedTaskException()); @@ -145,7 +145,7 @@ public class DiagnosticEventsTest { assertEquals(rejectedTaskEvent.getExecutorStateEvent().getLargestPoolSize(), largestPoolSize); assertEquals(rejectedTaskEvent.getExecutorStateEvent().getMaximumPoolSize(), maximumPoolSize); assertEquals(rejectedTaskEvent.getExecutorStateEvent().getLeasesOwned(), leaseAssignments.size()); - assertEquals(rejectedTaskEvent.getExecutorStateEvent().getCurrentQueueSize(),0); + assertEquals(0, rejectedTaskEvent.getExecutorStateEvent().getCurrentQueueSize()); assertTrue(rejectedTaskEvent.getThrowable() instanceof TestRejectedTaskException); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java index f7492d8d..71375c3d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java @@ -210,7 +210,7 @@ public class PeriodicShardSyncManagerTest { }}.stream().map(hashKeyRangeForLease -> { MultiStreamLease lease = new MultiStreamLease(); lease.hashKeyRange(hashKeyRangeForLease); - if(lease.hashKeyRangeForLease().startingHashKey().toString().equals("4")) { + if (lease.hashKeyRangeForLease().startingHashKey().toString().equals("4")) { lease.checkpoint(ExtendedSequenceNumber.SHARD_END); } else { lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); @@ -342,7 +342,7 @@ public class PeriodicShardSyncManagerTest { lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0]))); lease.shardId("shard-"+(leaseCounter[0])); // Setting the hashrange only for last two leases - if(leaseCounter[0] >= 3) { + if (leaseCounter[0] >= 3) { lease.hashKeyRange(hashKeyRangeForLease); } lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); @@ -355,7 +355,7 @@ public class PeriodicShardSyncManagerTest { Assert.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); // Assert that all the leases now has hashRanges set. - for(Lease lease : multiStreamLeases) { + for (Lease lease : multiStreamLeases) { Assert.assertNotNull(lease.hashKeyRangeForLease()); } } @@ -390,7 +390,7 @@ public class PeriodicShardSyncManagerTest { lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0]))); lease.shardId("shard-"+(leaseCounter[0])); // Setting the hashrange only for last two leases - if(leaseCounter[0] >= 3) { + if (leaseCounter[0] >= 3) { lease.hashKeyRange(hashKeyRangeForLease); } lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); @@ -403,14 +403,14 @@ public class PeriodicShardSyncManagerTest { Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); // Assert that all the leases now has hashRanges set. - for(Lease lease : multiStreamLeases) { + for (Lease lease : multiStreamLeases) { Assert.assertNotNull(lease.hashKeyRangeForLease()); } } @Test public void testFor1000DifferentValidSplitHierarchyTreeTheHashRangesAreAlwaysComplete() { - for(int i=0; i < 1000; i++) { + for (int i=0; i < 1000; i++) { int maxInitialLeaseCount = 100; List leases = generateInitialLeases(maxInitialLeaseCount); reshard(leases, 5, ReshardType.SPLIT, maxInitialLeaseCount, false); @@ -514,7 +514,7 @@ public class PeriodicShardSyncManagerTest { for (int i = 0; i < leasesToMerge; i += 2) { Lease parent1 = leasesEligibleForMerge.get(i); Lease parent2 = leasesEligibleForMerge.get(i + 1); - if(parent2.hashKeyRangeForLease().startingHashKey().subtract(parent1.hashKeyRangeForLease().endingHashKey()).equals(BigInteger.ONE)) + if (parent2.hashKeyRangeForLease().startingHashKey().subtract(parent1.hashKeyRangeForLease().endingHashKey()).equals(BigInteger.ONE)) { parent1.checkpoint(ExtendedSequenceNumber.SHARD_END); if (!shouldKeepSomeParentsInProgress || (shouldKeepSomeParentsInProgress && isOneFromDiceRoll())) { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 46918f62..3b5bfec9 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -35,7 +35,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.atMost; -import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.*; import java.time.Duration; import java.util.ArrayList; @@ -104,6 +103,9 @@ import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.Checkpointer; import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.AutoDetectionAndDeferredDeletionStrategy; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.NoLeaseDeletionStrategy; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.ProvidedStreamsDeferredDeletionStrategy; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; @@ -727,8 +729,8 @@ public class SchedulerTest { boolean expectPendingStreamsForDeletion, boolean onlyStreamsNoLeasesDeletion) throws DependencyException, ProvisionedThroughputException, InvalidStateException { - List streamConfigList1 = createDummyStreamConfigList(1,5); - List streamConfigList2 = createDummyStreamConfigList(3,7); + List streamConfigList1 = createDummyStreamConfigList(1, 5); + List streamConfigList2 = createDummyStreamConfigList(3, 7); retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) .retrievalFactory(retrievalFactory); when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); @@ -742,7 +744,7 @@ public class SchedulerTest { Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) .collect(Collectors.toCollection(HashSet::new)); - if(onlyStreamsNoLeasesDeletion) { + if (onlyStreamsNoLeasesDeletion) { expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) @@ -756,7 +758,7 @@ public class SchedulerTest { Assert.assertEquals(expectedSyncedStreams, syncedStreams); List expectedCurrentStreamConfigs; - if(onlyStreamsNoLeasesDeletion) { + if (onlyStreamsNoLeasesDeletion) { expectedCurrentStreamConfigs = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), @@ -778,8 +780,8 @@ public class SchedulerTest { @Test public void testKinesisStaleDeletedStreamCleanup() throws ProvisionedThroughputException, InvalidStateException, DependencyException { - List streamConfigList1 = createDummyStreamConfigList(1,6); - List streamConfigList2 = createDummyStreamConfigList(1,4); + List streamConfigList1 = createDummyStreamConfigList(1, 6); + List streamConfigList2 = createDummyStreamConfigList(1, 4); prepareForStaleDeletedStreamCleanupTests(streamConfigList1, streamConfigList2); @@ -820,7 +822,7 @@ public class SchedulerTest { @Test public void testKinesisStaleDeletedStreamNoCleanUpForTrackedStream() throws ProvisionedThroughputException, InvalidStateException, DependencyException { - List streamConfigList1 = createDummyStreamConfigList(1,6); + List streamConfigList1 = createDummyStreamConfigList(1, 6); prepareForStaleDeletedStreamCleanupTests(streamConfigList1); scheduler.deletedStreamListProvider().add(createDummyStreamConfig(3).streamIdentifier()); @@ -1243,7 +1245,7 @@ public class SchedulerTest { @Override public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig, DeletedStreamListProvider deletedStreamListProvider) { - if(shouldReturnDefaultShardSyncTaskmanager) { + if (shouldReturnDefaultShardSyncTaskmanager) { return shardSyncTaskManager; } final ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); @@ -1255,7 +1257,7 @@ public class SchedulerTest { when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(hierarchicalShardSyncer); when(shardDetector.streamIdentifier()).thenReturn(streamConfig.streamIdentifier()); when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); - if(shardSyncFirstAttemptFailure) { + if (shardSyncFirstAttemptFailure) { when(shardDetector.listShards()) .thenThrow(new RuntimeException("Service Exception")) .thenReturn(Collections.EMPTY_LIST); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java index 11d17368..17cad629 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java @@ -118,8 +118,7 @@ public class WorkerTest { private static final IRecordProcessorFactory SAMPLE_RECORD_PROCESSOR_FACTORY_V2 = SAMPLE_RECORD_PROCESSOR_FACTORY; - - *//** + *//* * Test method for {@link Worker#getApplicationName()}. *//* @Test @@ -346,7 +345,7 @@ public class WorkerTest { Assert.assertTrue(count > 0); } - *//** + *//* * Runs worker with threadPoolSize == numShards * Test method for {@link Worker#run()}. *//* @@ -357,7 +356,7 @@ public class WorkerTest { runAndTestWorker(numShards, threadPoolSize); } - *//** + *//* * Runs worker with threadPoolSize < numShards * Test method for {@link Worker#run()}. *//* @@ -368,7 +367,7 @@ public class WorkerTest { runAndTestWorker(numShards, threadPoolSize); } - *//** + *//* * Runs worker with threadPoolSize > numShards * Test method for {@link Worker#run()}. *//* @@ -379,7 +378,7 @@ public class WorkerTest { runAndTestWorker(numShards, threadPoolSize); } - *//** + *//* * Runs worker with threadPoolSize < numShards * Test method for {@link Worker#run()}. *//* @@ -395,7 +394,7 @@ public class WorkerTest { runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config); } - *//** + *//* * Runs worker with threadPoolSize < numShards * Test method for {@link Worker#run()}. *//* @@ -557,7 +556,7 @@ public class WorkerTest { verify(v2RecordProcessor, times(1)).shutdown(any(ShutdownInput.class)); } - *//** + *//* * This test is testing the {@link Worker}'s shutdown behavior and by extension the behavior of * {@link ThreadPoolExecutor#shutdownNow()}. It depends on the thread pool sending an interrupt to the pool threads. * This behavior makes the test a bit racy, since we need to ensure a specific order of events. @@ -1734,7 +1733,8 @@ public class WorkerTest { return new ReflectionFieldMatcher<>(itemClass, fieldName, fieldMatcher); } } - *//** + + *//* * Returns executor service that will be owned by the worker. This is useful to test the scenario * where worker shuts down the executor service also during shutdown flow. * @@ -1756,9 +1756,6 @@ public class WorkerTest { return shards; } - *//** - * @return - *//* private List createShardListWithOneSplit() { List shards = new ArrayList(); SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("39428", "987324"); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index 9e130c38..1a1abc0e 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -1592,7 +1592,7 @@ public class HierarchicalShardSyncerTest { assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON); } - /** + /* *
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
@@ -1869,7 +1869,7 @@ public class HierarchicalShardSyncerTest {
         assertExpectedLeasesAreCreated(SHARD_GRAPH_A, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP);
     }
 
-    /**
+    /*
      * 
      * Shard structure (x-axis is epochs):
      * 0  3   6   9
@@ -2325,12 +2325,16 @@ public class HierarchicalShardSyncerTest {
     @Test
     public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRangeAfterTwoRetries() throws Exception {
         final List shardsWithIncompleteHashRange = Arrays.asList(
-                ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "69")),
-                ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("71", ShardObjectHelper.MAX_HASH_KEY))
+                ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "69")),
+                ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange("71", ShardObjectHelper.MAX_HASH_KEY))
         );
         final List shardsWithCompleteHashRange = Arrays.asList(
-                ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")),
-                ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY))
+                ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")),
+                ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY))
         );
 
         when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true);
@@ -2352,8 +2356,10 @@ public class HierarchicalShardSyncerTest {
     @Test
     public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRange() throws Exception {
         final List shardsWithCompleteHashRange = Arrays.asList(
-                ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")),
-                ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY))
+                ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")),
+                ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY))
         );
 
         when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java
index 186fe290..72b48f16 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCoordinatorExerciser.java
@@ -14,10 +14,11 @@
  */
 package software.amazon.kinesis.leases;
 
-import java.awt.*;
+import java.awt.Button;
+import java.awt.Dimension;
+import java.awt.GridLayout;
 import java.awt.event.ActionEvent;
 import java.awt.event.ActionListener;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -25,7 +26,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import javax.swing.*;
+import javax.swing.BoxLayout;
+import javax.swing.JFrame;
+import javax.swing.JLabel;
+import javax.swing.JPanel;
 
 import lombok.extern.slf4j.Slf4j;
 import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
@@ -54,9 +58,8 @@ public class LeaseCoordinatorExerciser {
     private static final long INITIAL_LEASE_TABLE_READ_CAPACITY = 10L;
     private static final long INITIAL_LEASE_TABLE_WRITE_CAPACITY = 50L;
 
-    public static void main(String[] args) throws InterruptedException, DependencyException, InvalidStateException,
-            ProvisionedThroughputException, IOException {
-
+    public static void main(String[] args) throws DependencyException, InvalidStateException,
+            ProvisionedThroughputException {
         int numCoordinators = 9;
         int numLeases = 73;
         int leaseDurationMillis = 10000;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java
index ee2504d8..cc03a203 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java
@@ -56,7 +56,6 @@ public class ShardObjectHelper {
     private ShardObjectHelper() {
     }
 
-
     /** Helper method to create a new shard object.
      * @param shardId
      * @param parentShardId
@@ -84,7 +83,9 @@ public class ShardObjectHelper {
                                  String adjacentParentShardId,
                                  SequenceNumberRange sequenceNumberRange,
                                  HashKeyRange hashKeyRange) {
-        return Shard.builder().shardId(shardId).parentShardId(parentShardId).adjacentParentShardId(adjacentParentShardId).sequenceNumberRange(sequenceNumberRange).hashKeyRange(hashKeyRange).build();
+        return Shard.builder().shardId(shardId).parentShardId(parentShardId)
+                .adjacentParentShardId(adjacentParentShardId).sequenceNumberRange(sequenceNumberRange)
+                .hashKeyRange(hashKeyRange).build();
     }
 
     /** Helper method.
@@ -116,5 +117,4 @@ public class ShardObjectHelper {
         return parentShardIds;
     }
 
-
 }
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerBillingModePayPerRequestIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerBillingModePayPerRequestIntegrationTest.java
index 1dad013e..3f692da5 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerBillingModePayPerRequestIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerBillingModePayPerRequestIntegrationTest.java
@@ -37,7 +37,7 @@ import static org.junit.Assert.assertThat;
 @RunWith(MockitoJUnitRunner.class)
 public class DynamoDBLeaseRenewerBillingModePayPerRequestIntegrationTest extends
         LeaseIntegrationBillingModePayPerRequestTest {
-    private final String TEST_METRIC = "TestOperation";
+    private static final String TEST_METRIC = "TestOperation";
 
     // This test case's leases last 2 seconds
     private static final long LEASE_DURATION_MILLIS = 2000L;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerIntegrationTest.java
index 7c884fd6..f179a073 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerIntegrationTest.java
@@ -36,7 +36,7 @@ import software.amazon.kinesis.metrics.NullMetricsFactory;
 import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
 @RunWith(MockitoJUnitRunner.class)
 public class DynamoDBLeaseRenewerIntegrationTest extends LeaseIntegrationTest {
-    private final String TEST_METRIC = "TestOperation";
+    private static final String TEST_METRIC = "TestOperation";
 
     // This test case's leases last 2 seconds
     private static final long LEASE_DURATION_MILLIS = 2000L;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java
index bfff4e92..72379e88 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java
@@ -86,7 +86,7 @@ public class DynamoDBLeaseRenewerTest {
          */
         Lease lease1 = newLease("1");
         Lease lease2 = newLease("2");
-        leasesToRenew = Arrays.asList(lease1,lease2);
+        leasesToRenew = Arrays.asList(lease1, lease2);
         renewer.addLeasesToRenew(leasesToRenew);
 
         doReturn(true).when(leaseRefresher).renewLease(lease1);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/MetricAccumulatingQueueTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/MetricAccumulatingQueueTest.java
index 18bba742..0354a214 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/MetricAccumulatingQueueTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/MetricAccumulatingQueueTest.java
@@ -47,8 +47,8 @@ public class MetricAccumulatingQueueTest {
      */
     @Test
     public void testAccumulation() {
-        Collection dimensionsA = Collections.singleton(dim("name","a"));
-        Collection dimensionsB = Collections.singleton(dim("name","b"));
+        Collection dimensionsA = Collections.singleton(dim("name", "a"));
+        Collection dimensionsB = Collections.singleton(dim("name", "b"));
         String keyA = "a";
         String keyB = "b";
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
index 40d86c49..0f8e628e 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
@@ -176,7 +176,7 @@ public class FanOutRecordsPublisherTest {
     }
 
     @Test
-    public void InvalidEventTest() throws Exception {
+    public void testInvalidEvent() {
         FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN);
 
         ArgumentCaptor captor = ArgumentCaptor
@@ -443,10 +443,11 @@ public class FanOutRecordsPublisherTest {
 
                     @Override public void onNext(RecordsRetrieved input) {
                         receivedInput.add(input.processRecordsInput());
-                        assertEquals("" + ++lastSeenSeqNum, ((FanOutRecordsPublisher.FanoutRecordsRetrieved)input).continuationSequenceNumber());
+                        assertEquals("" + ++lastSeenSeqNum,
+                                ((FanOutRecordsPublisher.FanoutRecordsRetrieved) input).continuationSequenceNumber());
                         subscription.request(1);
                         servicePublisher.request(1);
-                        if(receivedInput.size() == totalServicePublisherEvents) {
+                        if (receivedInput.size() == totalServicePublisherEvents) {
                             servicePublisherTaskCompletionLatch.countDown();
                         }
                     }
@@ -549,10 +550,11 @@ public class FanOutRecordsPublisherTest {
 
                     @Override public void onNext(RecordsRetrieved input) {
                         receivedInput.add(input.processRecordsInput());
-                        assertEquals("" + ++lastSeenSeqNum, ((FanOutRecordsPublisher.FanoutRecordsRetrieved)input).continuationSequenceNumber());
+                        assertEquals("" + ++lastSeenSeqNum,
+                                ((FanOutRecordsPublisher.FanoutRecordsRetrieved) input).continuationSequenceNumber());
                         subscription.request(1);
                         servicePublisher.request(1);
-                        if(receivedInput.size() == triggerCompleteAtNthEvent) {
+                        if (receivedInput.size() == triggerCompleteAtNthEvent) {
                             servicePublisherTaskCompletionLatch.countDown();
                         }
                     }
@@ -681,7 +683,7 @@ public class FanOutRecordsPublisherTest {
                         receivedInput.add(input.processRecordsInput());
                         subscription.request(1);
                         servicePublisher.request(1);
-                        if(receivedInput.size() == triggerCompleteAtNthEvent) {
+                        if (receivedInput.size() == triggerCompleteAtNthEvent) {
                             servicePublisherTaskCompletionLatch.countDown();
                         }
                     }
@@ -783,10 +785,11 @@ public class FanOutRecordsPublisherTest {
 
                     @Override public void onNext(RecordsRetrieved input) {
                         receivedInput.add(input.processRecordsInput());
-                        assertEquals("" + ++lastSeenSeqNum, ((FanOutRecordsPublisher.FanoutRecordsRetrieved)input).continuationSequenceNumber());
+                        assertEquals("" + ++lastSeenSeqNum,
+                                ((FanOutRecordsPublisher.FanoutRecordsRetrieved) input).continuationSequenceNumber());
                         subscription.request(1);
                         servicePublisher.request(1);
-                        if(receivedInput.size() == triggerErrorAtNthEvent) {
+                        if (receivedInput.size() == triggerErrorAtNthEvent) {
                             servicePublisherTaskCompletionLatch.countDown();
                         }
                     }
@@ -879,10 +882,11 @@ public class FanOutRecordsPublisherTest {
 
                     @Override public void onNext(RecordsRetrieved input) {
                         receivedInput.add(input.processRecordsInput());
-                        assertEquals("" + ++lastSeenSeqNum, ((FanOutRecordsPublisher.FanoutRecordsRetrieved)input).continuationSequenceNumber());
+                        assertEquals("" + ++lastSeenSeqNum,
+                                ((FanOutRecordsPublisher.FanoutRecordsRetrieved) input).continuationSequenceNumber());
                         subscription.request(1);
                         servicePublisher.request(1);
-                        if(receivedInput.size() == totalServicePublisherEvents) {
+                        if (receivedInput.size() == totalServicePublisherEvents) {
                             servicePublisherTaskCompletionLatch.countDown();
                         }
                     }
@@ -973,7 +977,8 @@ public class FanOutRecordsPublisherTest {
 
                     @Override public void onNext(RecordsRetrieved input) {
                         receivedInput.add(input.processRecordsInput());
-                        assertEquals("" + ++lastSeenSeqNum, ((FanOutRecordsPublisher.FanoutRecordsRetrieved)input).continuationSequenceNumber());
+                        assertEquals("" + ++lastSeenSeqNum,
+                                ((FanOutRecordsPublisher.FanoutRecordsRetrieved) input).continuationSequenceNumber());
                         subscription.request(1);
                         servicePublisher.request(1);
                     }
@@ -1328,7 +1333,7 @@ public class FanOutRecordsPublisherTest {
                 fanOutRecordsPublisher
                         .evictAckedEventAndScheduleNextEvent(() -> recordsRetrieved.batchUniqueIdentifier());
                 // Send stale event periodically
-                if(totalRecordsRetrieved[0] % 10 == 0) {
+                if (totalRecordsRetrieved[0] % 10 == 0) {
                     fanOutRecordsPublisher.evictAckedEventAndScheduleNextEvent(
                             () -> new BatchUniqueIdentifier("some_uuid_str", "some_old_flow"));
                 }
@@ -1368,7 +1373,7 @@ public class FanOutRecordsPublisherTest {
         int count = 0;
         // Now that we allowed upto 10 elements queued up, send a pair of good and stale ack to verify records
         // delivered as expected.
-        while(count++ < 10 && (batchUniqueIdentifierQueued = ackQueue.take()) != null) {
+        while (count++ < 10 && (batchUniqueIdentifierQueued = ackQueue.take()) != null) {
             final BatchUniqueIdentifier batchUniqueIdentifierFinal = batchUniqueIdentifierQueued;
             fanOutRecordsPublisher
                     .evictAckedEventAndScheduleNextEvent(() -> batchUniqueIdentifierFinal);
@@ -1403,7 +1408,7 @@ public class FanOutRecordsPublisherTest {
         int count = 0;
         // Now that we allowed upto 10 elements queued up, send a pair of good and stale ack to verify records
         // delivered as expected.
-        while(count++ < 2 && (batchUniqueIdentifierQueued = ackQueue.poll(1000, TimeUnit.MILLISECONDS)) != null) {
+        while (count++ < 2 && (batchUniqueIdentifierQueued = ackQueue.poll(1000, TimeUnit.MILLISECONDS)) != null) {
             final BatchUniqueIdentifier batchUniqueIdentifierFinal = batchUniqueIdentifierQueued;
             fanOutRecordsPublisher.evictAckedEventAndScheduleNextEvent(
                     () -> new BatchUniqueIdentifier("some_uuid_str", batchUniqueIdentifierFinal.getFlowIdentifier()));
@@ -1457,7 +1462,8 @@ public class FanOutRecordsPublisherTest {
 
         flowCaptor.getValue().exceptionOccurred(exception);
 
-        Optional onErrorEvent = subscriber.events.stream().filter(e -> e instanceof OnErrorEvent).map(e -> (OnErrorEvent)e).findFirst();
+        Optional onErrorEvent = subscriber.events.stream().filter(e -> e instanceof OnErrorEvent)
+                .map(e -> (OnErrorEvent) e).findFirst();
 
         assertThat(onErrorEvent, equalTo(Optional.of(new OnErrorEvent(exception))));
         assertThat(acquireTimeoutLogged.get(), equalTo(true));
@@ -1587,8 +1593,8 @@ public class FanOutRecordsPublisherTest {
         public void run() {
             for (int i = 1; i <= numOfTimes; ) {
                 demandNotifier.acquireUninterruptibly();
-                if(i == sendCompletionAt) {
-                    if(shardEndAction != null) {
+                if (i == sendCompletionAt) {
+                    if (shardEndAction != null) {
                         shardEndAction.accept(i++);
                     } else {
                         action.accept(i++);
@@ -1596,7 +1602,7 @@ public class FanOutRecordsPublisherTest {
                     completeAction.run();
                     break;
                 }
-                if(i == sendErrorAt) {
+                if (i == sendErrorAt) {
                     action.accept(i++);
                     errorAction.run();
                     break;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java
index 2e09f34a..4ac8bbf7 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java
@@ -331,7 +331,7 @@ public class KinesisDataFetcherTest {
 
     private CompletableFuture makeGetRecordsResponse(String nextIterator, List records) {
         List childShards = new ArrayList<>();
-        if(nextIterator == null) {
+        if (nextIterator == null) {
             childShards = createChildShards();
         }
         return CompletableFuture.completedFuture(GetRecordsResponse.builder().nextShardIterator(nextIterator)
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java
index 5d757a6c..d9955da4 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
-import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
@@ -277,7 +276,8 @@ public class PrefetchRecordsPublisherIntegrationTest {
 
         @Override
         public DataFetcherResult getRecords() {
-            GetRecordsResponse getRecordsResult = GetRecordsResponse.builder().records(new ArrayList<>(records)).nextShardIterator(nextShardIterator).millisBehindLatest(1000L).build();
+            GetRecordsResponse getRecordsResult = GetRecordsResponse.builder().records(new ArrayList<>(records))
+                    .nextShardIterator(nextShardIterator).millisBehindLatest(1000L).build();
 
             return new AdvancingResult(getRecordsResult);
         }
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java
index 74707eb4..af02469a 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java
@@ -327,7 +327,7 @@ public class PrefetchRecordsPublisherTest {
         //        TODO: fix this verification
         //        verify(getRecordsRetrievalStrategy, times(callRate)).getRecords(MAX_RECORDS_PER_CALL);
         //        assertEquals(spyQueue.size(), callRate);
-        assertTrue("Call Rate is "+callRate,callRate < MAX_SIZE);
+        assertTrue("Call Rate is " + callRate, callRate < MAX_SIZE);
     }
 
     @Test
@@ -422,8 +422,10 @@ public class PrefetchRecordsPublisherTest {
 
     @Test
     public void testRetryableRetrievalExceptionContinues() {
-        GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L).records(Collections.emptyList()).nextShardIterator(NEXT_SHARD_ITERATOR).build();
-        when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenThrow(new RetryableRetrievalException("Timeout", new TimeoutException("Timeout"))).thenReturn(response);
+        GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L)
+                .records(Collections.emptyList()).nextShardIterator(NEXT_SHARD_ITERATOR).build();
+        when(getRecordsRetrievalStrategy.getRecords(anyInt()))
+                .thenThrow(new RetryableRetrievalException("Timeout", new TimeoutException("Timeout"))).thenReturn(response);
 
         getRecordsCache.start(sequenceNumber, initialPosition);
 
@@ -638,7 +640,7 @@ public class PrefetchRecordsPublisherTest {
 
         verify(getRecordsRetrievalStrategy, atLeast(2)).getRecords(anyInt());
 
-        while(getRecordsCache.getPublisherSession().prefetchRecordsQueue().remainingCapacity() > 0) {
+        while (getRecordsCache.getPublisherSession().prefetchRecordsQueue().remainingCapacity() > 0) {
             Thread.yield();
         }
 
@@ -697,7 +699,7 @@ public class PrefetchRecordsPublisherTest {
 
         public void resetIteratorTo(String nextIterator) {
             Iterator newIterator = responses.iterator();
-            while(newIterator.hasNext()) {
+            while (newIterator.hasNext()) {
                 GetRecordsResponse current = newIterator.next();
                 if (StringUtils.equals(nextIterator, current.nextShardIterator())) {
                     if (!newIterator.hasNext()) {
@@ -725,7 +727,7 @@ public class PrefetchRecordsPublisherTest {
 
         private static final int LOSS_EVERY_NTH_RECORD = 50;
         private static int recordCounter = 0;
-        private static final ScheduledExecutorService consumerHealthChecker = Executors.newScheduledThreadPool(1);
+        private static final ScheduledExecutorService CONSUMER_HEALTH_CHECKER = Executors.newScheduledThreadPool(1);
 
         public LossyNotificationSubscriber(Subscriber delegate, RecordsPublisher recordsPublisher) {
             super(delegate, recordsPublisher);
@@ -738,7 +740,7 @@ public class PrefetchRecordsPublisherTest {
                 getDelegateSubscriber().onNext(recordsRetrieved);
             } else {
                 log.info("Record Loss Triggered");
-                consumerHealthChecker.schedule(() ->  {
+                CONSUMER_HEALTH_CHECKER.schedule(() ->  {
                     getRecordsPublisher().restartFrom(recordsRetrieved);
                     Flowable.fromPublisher(getRecordsPublisher()).subscribeOn(Schedulers.computation())
                             .observeOn(Schedulers.computation(), true, 8).subscribe(this);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java
index 0d68e51b..cd7ad8a6 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java
@@ -21,7 +21,7 @@ public class BlockingUtils {
 
     public static  Records blockUntilRecordsAvailable(Supplier recordsSupplier, long timeoutMillis) {
         Records recordsRetrieved;
-        while((recordsRetrieved = recordsSupplier.get()) == null && timeoutMillis > 0 ) {
+        while ((recordsRetrieved = recordsSupplier.get()) == null && timeoutMillis > 0 ) {
             try {
                 Thread.sleep(100);
             } catch (InterruptedException e) {
@@ -29,7 +29,7 @@ public class BlockingUtils {
             }
             timeoutMillis -= 100;
         }
-        if(recordsRetrieved != null) {
+        if (recordsRetrieved != null) {
             return recordsRetrieved;
         } else {
             throw new RuntimeException("No records found");
@@ -37,7 +37,7 @@ public class BlockingUtils {
     }
 
     public static boolean blockUntilConditionSatisfied(Supplier conditionSupplier, long timeoutMillis) {
-        while(!conditionSupplier.get() && timeoutMillis > 0 ) {
+        while (!conditionSupplier.get() && timeoutMillis > 0 ) {
             try {
                 Thread.sleep(100);
             } catch (InterruptedException e) {
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/SubscribeToShardRequestMatcher.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/SubscribeToShardRequestMatcher.java
index d120d95a..43c887a3 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/SubscribeToShardRequestMatcher.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/SubscribeToShardRequestMatcher.java
@@ -12,7 +12,7 @@ public class SubscribeToShardRequestMatcher extends ArgumentMatcher
         
         
-        
+        
+            
+            
+        
         
         
         
diff --git a/pom.xml b/pom.xml
index 82d7857d..90cec514 100644
--- a/pom.xml
+++ b/pom.xml
@@ -81,6 +81,7 @@
           checkstyle/checkstyle.xml
           true
           true
+          true
           checkstyle/checkstyle-suppressions.xml
         
         

From 768f6a36bb5755df6f95414922061f05d5f93f55 Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Mon, 26 Jun 2023 16:19:30 -0400
Subject: [PATCH 74/88] Checkstyle: added `UnusedImports` check. (#1153)

---
 .../MultiLangRecordProcessorFactory.java      |  1 -
 ...edentialsProviderPropertyValueDecoder.java |  2 --
 .../multilang/config/BuilderDynaBean.java     |  2 --
 .../config/KinesisClientLibConfigurator.java  |  6 +----
 .../config/MultiLangDaemonConfiguration.java  |  2 --
 .../kinesis/multilang/MessageWriterTest.java  |  2 --
 .../multilang/MultiLangProtocolTest.java      |  9 ++++----
 .../kinesis/leases/MultiStreamLease.java      |  2 --
 .../kinesis/metrics/MetricDatumWithKey.java   |  7 +++---
 .../fanout/FanOutRecordsPublisher.java        |  4 ----
 .../checkpoint/InMemoryCheckpointer.java      |  1 -
 .../amazon/kinesis/config/KCLAppConfig.java   |  2 --
 .../GracefulShutdownCoordinatorTest.java      |  2 --
 .../kinesis/coordinator/WorkerTest.java       |  2 --
 ...tegrationBillingModePayPerRequestTest.java |  8 -------
 .../amazon/kinesis/leases/ShardInfoTest.java  |  2 --
 ...namoDBLeaseCoordinatorIntegrationTest.java |  1 -
 .../dynamodb/DynamoDBLeaseRefresherTest.java  |  2 --
 .../DynamoDBLeaseTakerIntegrationTest.java    |  3 ---
 .../lifecycle/BlockOnParentShardTaskTest.java |  6 +----
 .../ShardConsumerSubscriberTest.java          | 22 -------------------
 .../FanOutConsumerRegistrationTest.java       |  2 --
 .../fanout/FanOutRecordsPublisherTest.java    | 18 +--------------
 .../kinesis/utils/AWSResourceManager.java     |  8 +------
 .../kinesis/utils/LeaseTableManager.java      |  2 --
 .../kinesis/utils/StreamExistenceManager.java |  2 --
 checkstyle/checkstyle.xml                     |  1 +
 27 files changed, 12 insertions(+), 109 deletions(-)

diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangRecordProcessorFactory.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangRecordProcessorFactory.java
index a1c01c51..c4aab958 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangRecordProcessorFactory.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangRecordProcessorFactory.java
@@ -19,7 +19,6 @@ import java.util.concurrent.ExecutorService;
 import com.fasterxml.jackson.databind.ObjectMapper;
 
 import lombok.extern.slf4j.Slf4j;
-import software.amazon.kinesis.coordinator.KinesisClientLibConfiguration;
 import software.amazon.kinesis.multilang.config.MultiLangDaemonConfiguration;
 import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
 import software.amazon.kinesis.processor.ShardRecordProcessor;
diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java
index da3db4fb..97fa975e 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java
@@ -22,8 +22,6 @@ import java.util.List;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.AWSCredentialsProviderChain;
 import lombok.extern.slf4j.Slf4j;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
-import software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain;
 
 /**
  * Get AWSCredentialsProvider property.
diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java
index 2e5502cd..2035695c 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java
@@ -20,7 +20,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Optional;
 import java.util.function.Function;
-import java.util.function.Supplier;
 
 import lombok.Getter;
 import org.apache.commons.beanutils.ConvertUtilsBean;
@@ -150,7 +149,6 @@ public class BuilderDynaBean implements DynaBean {
         } else {
             return expected.cast(dynaBeanCreateSupport.build());
         }
-
     }
 
     private void validateResolvedEmptyHandler() {
diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java
index c95d0853..5e2ddb1d 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java
@@ -25,7 +25,6 @@ import org.apache.commons.beanutils.ConvertUtilsBean;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.commons.lang3.Validate;
 import software.amazon.awssdk.arns.Arn;
-import software.amazon.awssdk.regions.Region;
 import software.amazon.kinesis.common.StreamIdentifier;
 
 /**
@@ -42,7 +41,6 @@ public class KinesisClientLibConfigurator {
     private final BeanUtilsBean utilsBean;
     private final MultiLangDaemonConfiguration configuration;
 
-
     /**
      * Constructor.
      */
@@ -78,7 +76,6 @@ public class KinesisClientLibConfigurator {
             //Parse out the stream Name from the Arn (and/or override existing value for Stream Name)
             final String streamNameFromArn = streamArnObj.resource().resource();
             configuration.setStreamName(streamNameFromArn);
-
         }
 
         Validate.notBlank(configuration.getStreamName(), "Stream name or Stream Arn is required. Stream Arn takes precedence if both are passed in.");
@@ -109,5 +106,4 @@ public class KinesisClientLibConfigurator {
         return getConfiguration(properties);
     }
 
-
-}
\ No newline at end of file
+}
diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java
index 7a7f2e79..d8f58741 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfiguration.java
@@ -158,7 +158,6 @@ public class MultiLangDaemonConfiguration {
         metricsEnabledDimensions = new HashSet<>(Arrays.asList(dimensions));
     }
 
-
     private RetrievalMode retrievalMode = RetrievalMode.DEFAULT;
 
     private final FanoutConfigBean fanoutConfig = new FanoutConfigBean();
@@ -170,7 +169,6 @@ public class MultiLangDaemonConfiguration {
     private long shutdownGraceMillis;
     private Integer timeoutInSeconds;
 
-
     private final BuilderDynaBean kinesisCredentialsProvider;
 
     public void setAWSCredentialsProvider(String providerString) {
diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java
index c997c193..588f6140 100644
--- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java
+++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MessageWriterTest.java
@@ -51,8 +51,6 @@ public class MessageWriterTest {
     @Rule
     public final ExpectedException thrown = ExpectedException.none();
 
-    // ExecutorService executor;
-
     @Before
     public void setup() {
         stream = Mockito.mock(OutputStream.class);
diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java
index d385b2f9..5320aec5 100644
--- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java
+++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangProtocolTest.java
@@ -103,7 +103,7 @@ public class MultiLangProtocolTest {
     }
 
     @Test
-    public void initializeTest() throws InterruptedException, ExecutionException {
+    public void testInitialize() {
         when(messageWriter
                 .writeInitializeMessage(argThat(Matchers.withInit(InitializationInput.builder()
                         .shardId(shardId).build())))).thenReturn(buildFuture(true));
@@ -113,7 +113,7 @@ public class MultiLangProtocolTest {
     }
 
     @Test
-    public void processRecordsTest() throws InterruptedException, ExecutionException {
+    public void testProcessRecords() {
         when(messageWriter.writeProcessRecordsMessage(any(ProcessRecordsInput.class))).thenReturn(buildFuture(true));
         when(messageReader.getNextMessageFromSTDOUT()).thenReturn(buildFuture(
                 new StatusMessage("processRecords"), Message.class));
@@ -128,7 +128,6 @@ public class MultiLangProtocolTest {
         when(messageReader.getNextMessageFromSTDOUT()).thenReturn(buildFuture(new StatusMessage(LeaseLostMessage.ACTION), Message.class));
 
         assertThat(protocol.leaseLost(LeaseLostInput.builder().build()), equalTo(true));
-
     }
 
     @Test
@@ -174,7 +173,7 @@ public class MultiLangProtocolTest {
     }
 
     @Test
-    public void processRecordsWithCheckpointsTest() throws InterruptedException, ExecutionException,
+    public void testProcessRecordsWithCheckpoints() throws
         KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
 
         when(messageWriter.writeProcessRecordsMessage(any(ProcessRecordsInput.class))).thenReturn(buildFuture(true));
@@ -203,7 +202,7 @@ public class MultiLangProtocolTest {
     }
 
     @Test
-    public void processRecordsWithABadCheckpointTest() throws InterruptedException, ExecutionException {
+    public void testProcessRecordsWithABadCheckpoint() {
         when(messageWriter.writeProcessRecordsMessage(any(ProcessRecordsInput.class))).thenReturn(buildFuture(true));
         when(messageWriter.writeCheckpointMessageWithError(anyString(), anyLong(), any(Throwable.class))).thenReturn(buildFuture(false));
         when(messageReader.getNextMessageFromSTDOUT()).thenAnswer(buildMessageAnswers(new ArrayList() {
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java
index c8811354..c79cc458 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java
@@ -23,8 +23,6 @@ import lombok.Setter;
 import lombok.experimental.Accessors;
 import org.apache.commons.lang3.Validate;
 
-import java.util.Objects;
-
 import static com.google.common.base.Verify.verifyNotNull;
 
 @Setter
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java
index da83675f..e94c8730 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricDatumWithKey.java
@@ -15,7 +15,6 @@
 package software.amazon.kinesis.metrics;
 
 import lombok.AllArgsConstructor;
-import lombok.Data;
 import lombok.Setter;
 import lombok.experimental.Accessors;
 import software.amazon.awssdk.services.cloudwatch.model.MetricDatum;
@@ -29,11 +28,11 @@ import java.util.Objects;
  * @param  is a class that stores information about a MetricDatum. This is useful
  *        to compare MetricDatums, aggregate similar MetricDatums or store information about a datum
  *        that may be relevant to the user (i.e. MetricName, CustomerId, TimeStamp, etc).
- * 
+ *
  *        Example:
- * 
+ *
  *        Let SampleMetricKey be a KeyType that takes in the time in which the datum was created.
- * 
+ *
  *        MetricDatumWithKey sampleDatumWithKey = new MetricDatumWithKey(new
  *        SampleMetricKey(System.currentTimeMillis()), datum)
  */
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java
index ca4ce12d..7177211f 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java
@@ -27,14 +27,12 @@ import org.reactivestreams.Subscriber;
 import org.reactivestreams.Subscription;
 import software.amazon.awssdk.core.async.SdkPublisher;
 import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
-import software.amazon.awssdk.services.kinesis.model.ChildShard;
 import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
 import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEvent;
 import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream;
 import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest;
 import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponse;
 import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler;
-import software.amazon.awssdk.utils.CollectionUtils;
 import software.amazon.awssdk.utils.Either;
 import software.amazon.kinesis.annotations.KinesisClientInternalApi;
 import software.amazon.kinesis.common.InitialPositionInStreamExtended;
@@ -117,7 +115,6 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
             this.currentSequenceNumber = extendedSequenceNumber.sequenceNumber();
             this.isFirstConnection = true;
         }
-
     }
 
     @Override
@@ -276,7 +273,6 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
         SubscriptionShutdownEvent(Runnable subscriptionShutdownAction, String eventIdentifier) {
             this(subscriptionShutdownAction, eventIdentifier, null);
         }
-
     }
 
     private boolean hasValidSubscriber() {
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java
index a2d83568..2a93e83d 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java
@@ -17,7 +17,6 @@ package software.amazon.kinesis.checkpoint;
 import java.util.HashMap;
 import java.util.Map;
 
-import software.amazon.kinesis.exceptions.KinesisClientLibException;
 import software.amazon.kinesis.processor.Checkpointer;
 import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java
index b67efa10..5365ca4f 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java
@@ -29,7 +29,6 @@ import java.io.IOException;
 import java.net.Inet4Address;
 import java.net.URISyntaxException;
 import java.net.UnknownHostException;
-import java.util.Optional;
 
 /**
  * Default configuration for a producer or consumer used in integration tests.
@@ -81,7 +80,6 @@ public abstract class KCLAppConfig {
     }
 
     public final KinesisAsyncClient buildAsyncKinesisClient() throws URISyntaxException, IOException {
-
         if (kinesisAsyncClient == null) {
             // Setup H2 client config.
             final NettyNioAsyncHttpClient.Builder builder = NettyNioAsyncHttpClient.builder()
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/GracefulShutdownCoordinatorTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/GracefulShutdownCoordinatorTest.java
index 84dcaa9c..44c7272c 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/GracefulShutdownCoordinatorTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/GracefulShutdownCoordinatorTest.java
@@ -15,12 +15,10 @@
 package software.amazon.kinesis.coordinator;
 
 import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.not;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java
index 17cad629..ec076e8d 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/WorkerTest.java
@@ -14,8 +14,6 @@
  */
 package software.amazon.kinesis.coordinator;
 
-import java.util.concurrent.ThreadPoolExecutor;
-
 /**
  * Unit tests of Worker.
  */
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseIntegrationBillingModePayPerRequestTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseIntegrationBillingModePayPerRequestTest.java
index 9f7735f9..128d347a 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseIntegrationBillingModePayPerRequestTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseIntegrationBillingModePayPerRequestTest.java
@@ -15,16 +15,8 @@
 package software.amazon.kinesis.leases;
 
 import lombok.extern.slf4j.Slf4j;
-import org.junit.Rule;
-import org.junit.rules.TestWatcher;
-import org.junit.runner.Description;
-import org.mockito.Mock;
-import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
-import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
 import software.amazon.awssdk.services.dynamodb.model.BillingMode;
 import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher;
-import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer;
-import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback;
 
 @Slf4j
 public class LeaseIntegrationBillingModePayPerRequestTest extends LeaseIntegrationTest {
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardInfoTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardInfoTest.java
index 276f6c25..4ccafe52 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardInfoTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardInfoTest.java
@@ -26,11 +26,9 @@ import java.util.List;
 import java.util.Set;
 import java.util.UUID;
 
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import software.amazon.kinesis.leases.ShardInfo;
 import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
 
 public class ShardInfoTest {
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java
index d89c010e..05d4ba74 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java
index 643cc99c..102a9f17 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherTest.java
@@ -26,7 +26,6 @@ import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -158,7 +157,6 @@ public class DynamoDBLeaseRefresherTest {
 
         verify(mockScanFuture, times(2)).get(anyLong(), any(TimeUnit.class));
         verify(dynamoDbClient, times(2)).scan(any(ScanRequest.class));
-
     }
 
     @Test
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java
index 475f1940..772aa542 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java
@@ -15,9 +15,7 @@
 package software.amazon.kinesis.leases.dynamodb;
 
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -153,7 +151,6 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest {
         assertThat(addedLeases.values().containsAll(allLeases), equalTo(true));
     }
 
-
     /**
      * Sets the leaseDurationMillis to 0, ensuring a get request to update the existing lease after computing
      * leases to take
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java
index 06a72230..61473833 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java
@@ -22,7 +22,6 @@ import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Optional;
 
 import org.junit.Before;
 import org.junit.Test;
@@ -43,7 +42,7 @@ public class BlockOnParentShardTaskTest {
     private final String shardId = "shardId-97";
     private final String streamId = "123:stream:146";
     private final String concurrencyToken = "testToken";
-    private final List emptyParentShardIds = new ArrayList();
+    private final List emptyParentShardIds = new ArrayList<>();
     private ShardInfo shardInfo;
 
     @Before
@@ -77,7 +76,6 @@ public class BlockOnParentShardTaskTest {
     @Test
     public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinished()
         throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-
         ShardInfo shardInfo = null;
         BlockOnParentShardTask task = null;
         String parent1ShardId = "shardId-1";
@@ -118,7 +116,6 @@ public class BlockOnParentShardTaskTest {
     @Test
     public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinishedMultiStream()
             throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-
         ShardInfo shardInfo = null;
         BlockOnParentShardTask task = null;
         String parent1LeaseKey = streamId + ":" + "shardId-1";
@@ -162,7 +159,6 @@ public class BlockOnParentShardTaskTest {
     @Test
     public final void testCallWhenParentsHaveNotFinished()
             throws DependencyException, InvalidStateException, ProvisionedThroughputException {
-
         ShardInfo shardInfo = null;
         BlockOnParentShardTask task = null;
         String parent1ShardId = "shardId-1";
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java
index 09ba6ec9..4299c163 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriberTest.java
@@ -35,7 +35,6 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Optional;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -173,7 +172,6 @@ public class ShardConsumerSubscriberTest {
         assertThat(subscriber.getAndResetDispatchFailure(), nullValue());
 
         verify(shardConsumer, times(20)).handleInput(argThat(eqProcessRecordsInput(processRecordsInput)), any(Subscription.class));
-
     }
 
     @Test
@@ -293,12 +291,10 @@ public class ShardConsumerSubscriberTest {
         assertThat(received.size(), equalTo(recordsPublisher.responses.size()));
         Stream.iterate(0, i -> i + 1).limit(received.size()).forEach(i -> assertThat(received.get(i),
                 eqProcessRecordsInput(recordsPublisher.responses.get(i).recordsRetrieved.processRecordsInput())));
-
     }
 
     @Test
     public void restartAfterRequestTimerExpiresWhenNotGettingRecordsAfterInitialization() throws Exception {
-
         executorService = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
                 .setNameFormat("test-" + testName.getMethodName() + "-%04d").setDaemon(true).build());
 
@@ -347,12 +343,10 @@ public class ShardConsumerSubscriberTest {
         assertThat(received.size(), equalTo(recordsPublisher.responses.size()));
         Stream.iterate(0, i -> i + 1).limit(received.size()).forEach(i -> assertThat(received.get(i),
                 eqProcessRecordsInput(recordsPublisher.responses.get(i).recordsRetrieved.processRecordsInput())));
-
     }
 
     @Test
     public void restartAfterRequestTimerExpiresWhenInitialTaskExecutionIsRejected() throws Exception {
-
         executorService = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
                 .setNameFormat("test-" + testName.getMethodName() + "-%04d").setDaemon(true).build());
 
@@ -405,7 +399,6 @@ public class ShardConsumerSubscriberTest {
         assertThat(received.size(), equalTo(recordsPublisher.responses.size()));
         Stream.iterate(0, i -> i + 1).limit(received.size()).forEach(i -> assertThat(received.get(i),
                 eqProcessRecordsInput(recordsPublisher.responses.get(i).recordsRetrieved.processRecordsInput())));
-
     }
 
     private Object directlyExecuteRunnable(InvocationOnMock invocation) {
@@ -623,8 +616,6 @@ public class ShardConsumerSubscriberTest {
 
     /**
      * Test to validate the warning message from ShardConsumer is not suppressed with the default configuration of 0
-     * 
-     * @throws Exception
      */
     @Test
     public void noLoggingSuppressionNeededOnHappyPathTest() {
@@ -648,8 +639,6 @@ public class ShardConsumerSubscriberTest {
 
     /**
      * Test to validate the warning message from ShardConsumer is not suppressed with the default configuration of 0
-     * 
-     * @throws Exception
      */
     @Test
     public void loggingNotSuppressedAfterTimeoutTest() {
@@ -677,8 +666,6 @@ public class ShardConsumerSubscriberTest {
     /**
      * Test to validate the warning message from ShardConsumer is successfully supressed if we only have intermittant
      * readTimeouts.
-     * 
-     * @throws Exception
      */
     @Test
     public void loggingSuppressedAfterIntermittentTimeoutTest() {
@@ -705,8 +692,6 @@ public class ShardConsumerSubscriberTest {
     /**
      * Test to validate the warning message from ShardConsumer is successfully logged if multiple sequential timeouts
      * occur.
-     * 
-     * @throws Exception
      */
     @Test
     public void loggingPartiallySuppressedAfterMultipleTimeoutTest() {
@@ -733,8 +718,6 @@ public class ShardConsumerSubscriberTest {
 
     /**
      * Test to validate the warning message from ShardConsumer is successfully logged if sequential timeouts occur.
-     * 
-     * @throws Exception
      */
     @Test
     public void loggingPartiallySuppressedAfterConsecutiveTimeoutTest() {
@@ -763,8 +746,6 @@ public class ShardConsumerSubscriberTest {
     /**
      * Test to validate the non-timeout warning message from ShardConsumer is not suppressed with the default
      * configuration of 0
-     * 
-     * @throws Exception
      */
     @Test
     public void loggingNotSuppressedOnNonReadTimeoutExceptionNotIgnoringReadTimeoutsExceptionTest() {
@@ -792,12 +773,9 @@ public class ShardConsumerSubscriberTest {
     /**
      * Test to validate the non-timeout warning message from ShardConsumer is not suppressed with 2 ReadTimeouts to
      * ignore
-     * 
-     * @throws Exception
      */
     @Test
     public void loggingNotSuppressedOnNonReadTimeoutExceptionIgnoringReadTimeoutsTest() {
-
         // We're not throwing a ReadTimeout, so no suppression is expected.
         // The test expects a non-ReadTimeout exception to be thrown on requests 3 and 5, and we expect logs on
         // each Non-ReadTimeout Exception, no matter what the number of ReadTimeoutsToIgnore we pass in,
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistrationTest.java
index 245e22d5..fca6799d 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistrationTest.java
@@ -19,7 +19,6 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
@@ -28,7 +27,6 @@ import static org.mockito.Mockito.when;
 import java.util.concurrent.CompletableFuture;
 
 import org.apache.commons.lang3.StringUtils;
-import org.hamcrest.Matchers;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
index 0f8e628e..9615794b 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
@@ -8,7 +8,6 @@ import io.reactivex.rxjava3.schedulers.Schedulers;
 import io.reactivex.rxjava3.subscribers.SafeSubscriber;
 import lombok.Data;
 import lombok.RequiredArgsConstructor;
-import lombok.Setter;
 import lombok.extern.slf4j.Slf4j;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
@@ -54,7 +53,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -77,7 +75,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.argThat;
-import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.never;
@@ -172,7 +169,6 @@ public class FanOutRecordsPublisherTest {
                 assertThat(clientRecordsList.get(i), matchers.get(i));
             }
         });
-
     }
 
     @Test
@@ -239,7 +235,6 @@ public class FanOutRecordsPublisherTest {
                 assertThat(clientRecordsList.get(i), matchers.get(i));
             }
         });
-
     }
 
     @Test
@@ -317,11 +312,10 @@ public class FanOutRecordsPublisherTest {
         });
 
         assertThat(source.getCurrentSequenceNumber(), equalTo("3000"));
-
     }
 
     @Test
-    public void testIfEventsAreNotDeliveredToShardConsumerWhenPreviousEventDeliveryTaskGetsRejected() throws Exception {
+    public void testIfEventsAreNotDeliveredToShardConsumerWhenPreviousEventDeliveryTaskGetsRejected() {
         FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN);
 
         ArgumentCaptor captor = ArgumentCaptor
@@ -395,7 +389,6 @@ public class FanOutRecordsPublisherTest {
         });
 
         assertThat(source.getCurrentSequenceNumber(), equalTo("1000"));
-
     }
 
     @Test
@@ -489,12 +482,10 @@ public class FanOutRecordsPublisherTest {
         });
 
         assertThat(source.getCurrentSequenceNumber(), equalTo(totalServicePublisherEvents + ""));
-
     }
 
     @Test
     public void testIfStreamOfEventsAndOnCompleteAreDeliveredInOrderWithBackpressureAdheringServicePublisher() throws Exception {
-
         CountDownLatch onS2SCallLatch = new CountDownLatch(2);
 
         doAnswer(new Answer() {
@@ -601,7 +592,6 @@ public class FanOutRecordsPublisherTest {
         // Let's wait for sometime to allow the publisher to re-subscribe
         onS2SCallLatch.await(5000, TimeUnit.MILLISECONDS);
         verify(kinesisClient, times(2)).subscribeToShard(any(SubscribeToShardRequest.class), flowCaptor.capture());
-
     }
 
     @Test
@@ -730,7 +720,6 @@ public class FanOutRecordsPublisherTest {
         // With shard end event, onComplete must be propagated to the subscriber.
         onCompleteLatch.await(5000, TimeUnit.MILLISECONDS);
         assertTrue("OnComplete should be triggered", isOnCompleteTriggered[0]);
-
     }
 
     @Test
@@ -834,7 +823,6 @@ public class FanOutRecordsPublisherTest {
         assertThat(source.getCurrentSequenceNumber(), equalTo(triggerErrorAtNthEvent + ""));
         onErrorReceiveLatch.await(5000, TimeUnit.MILLISECONDS);
         assertTrue("OnError should have been thrown", isOnErrorThrown[0]);
-
     }
 
     @Test
@@ -928,7 +916,6 @@ public class FanOutRecordsPublisherTest {
         });
 
         assertThat(source.getCurrentSequenceNumber(), equalTo(totalServicePublisherEvents + ""));
-
     }
 
     @Test
@@ -1131,7 +1118,6 @@ public class FanOutRecordsPublisherTest {
                 assertThat(clientRecordsList.get(i), matchers.get(i));
             }
         });
-
     }
 
     @Test
@@ -1247,7 +1233,6 @@ public class FanOutRecordsPublisherTest {
 
         verifyRecords(nonFailingSubscriber.received.get(0).records(), matchers);
         verifyRecords(nonFailingSubscriber.received.get(1).records(), nextMatchers);
-
     }
 
     @Test
@@ -1467,7 +1452,6 @@ public class FanOutRecordsPublisherTest {
 
         assertThat(onErrorEvent, equalTo(Optional.of(new OnErrorEvent(exception))));
         assertThat(acquireTimeoutLogged.get(), equalTo(true));
-
     }
 
     private void verifyRecords(List clientRecordsList, List matchers) {
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java
index 3314f922..4a6bcfaf 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java
@@ -2,14 +2,8 @@ package software.amazon.kinesis.utils;
 
 import lombok.NoArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
-import software.amazon.awssdk.services.dynamodb.model.ListTablesRequest;
-import software.amazon.awssdk.services.dynamodb.model.ListTablesResponse;
-import software.amazon.kinesis.common.FutureUtils;
 
-import java.time.Duration;
-import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 @Slf4j
@@ -28,7 +22,7 @@ public abstract class AWSResourceManager {
 
     /**
      * Get a list of all the names of resources of a specified type
-     * @return
+     *
      * @throws Exception
      */
     public abstract List getAllResourceNames() throws Exception;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java
index e8d1cb05..40d711bd 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/LeaseTableManager.java
@@ -12,8 +12,6 @@ import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException;
 import software.amazon.awssdk.services.dynamodb.model.TableStatus;
 import software.amazon.kinesis.common.FutureUtils;
 
-import java.io.IOException;
-import java.net.URISyntaxException;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.List;
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java
index eeffb36b..b5f06b78 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java
@@ -2,8 +2,6 @@ package software.amazon.kinesis.utils;
 
 import lombok.Value;
 import lombok.extern.slf4j.Slf4j;
-import software.amazon.awssdk.services.dynamodb.model.ListTablesRequest;
-import software.amazon.awssdk.services.dynamodb.model.ListTablesResponse;
 import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
 import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest;
 import software.amazon.awssdk.services.kinesis.model.DeleteStreamRequest;
diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml
index 0186a496..10031e23 100644
--- a/checkstyle/checkstyle.xml
+++ b/checkstyle/checkstyle.xml
@@ -36,6 +36,7 @@
         
         
         
+        
         
     
 

From 4eff3981474c706350bd9f4f7c8a88e9b2501e9e Mon Sep 17 00:00:00 2001
From: pelaezryan 
Date: Tue, 27 Jun 2023 10:21:49 -0700
Subject: [PATCH 75/88] Preparation for v2.5.1 (#1155)

* Preparation for minor version v2.5.1

---------

Co-authored-by: Ryan Pelaez 
---
 CHANGELOG.md                                          | 11 +++++++++++
 README.md                                             |  2 +-
 amazon-kinesis-client-multilang/pom.xml               |  2 +-
 amazon-kinesis-client/pom.xml                         |  2 +-
 .../amazon/kinesis/retrieval/RetrievalConfig.java     |  2 +-
 pom.xml                                               |  2 +-
 6 files changed, 16 insertions(+), 5 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 54a7892e..fc7ec987 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -3,6 +3,17 @@
 For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md)
 
 ---
+### Release 2.5.1 (June 27, 2023)
+* [#1143](https://github.com/awslabs/amazon-kinesis-client/pull/1143) Upgrade MultiLangDaemon to support StreamARN
+* [#1145](https://github.com/awslabs/amazon-kinesis-client/pull/1145) Introduced GitHub actions to trigger Maven builds during merge/pull requests
+* [#1136](https://github.com/awslabs/amazon-kinesis-client/pull/1136) Added testing architecture and KCL 2.x basic polling/streaming tests
+* [#1153](https://github.com/awslabs/amazon-kinesis-client/pull/1153) Checkstyle: added `UnusedImports` check.
+* [#1150](https://github.com/awslabs/amazon-kinesis-client/pull/1150) Enabled Checkstyle validation of test resources.
+* [#1149](https://github.com/awslabs/amazon-kinesis-client/pull/1149) Bound Checkstyle to `validate` goal for automated enforcement.
+* [#1148](https://github.com/awslabs/amazon-kinesis-client/pull/1148) Code cleanup to faciliate Checkstyle enforcement.
+* [#1142](https://github.com/awslabs/amazon-kinesis-client/pull/1142) Upgrade Google Guava dependency version from 31.1-jre to 32.0.0-jre
+* [#1115](https://github.com/awslabs/amazon-kinesis-client/pull/1115) Update KCL version from 2.5.0 to 2.5.1-SNAPSHOT
+
 ### Release 2.5.0 (May 19, 2023)
 * **[#1109](https://github.com/awslabs/amazon-kinesis-client/pull/1109) Add support for stream ARNs**
 * **[#1065](https://github.com/awslabs/amazon-kinesis-client/pull/1065) Allow tags to be added when lease table is created**
diff --git a/README.md b/README.md
index 6328e115..3cc92799 100644
--- a/README.md
+++ b/README.md
@@ -56,7 +56,7 @@ The recommended way to use the KCL for Java is to consume it from Maven.
   
       software.amazon.kinesis
       amazon-kinesis-client
-      2.4.8
+      2.5.1
   
   ```
 
diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml
index b3a48210..8494c3c8 100644
--- a/amazon-kinesis-client-multilang/pom.xml
+++ b/amazon-kinesis-client-multilang/pom.xml
@@ -21,7 +21,7 @@
   
     amazon-kinesis-client-pom
     software.amazon.kinesis
-    2.5.1-SNAPSHOT
+    2.5.1
   
   4.0.0
 
diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml
index 30824281..2ba3aaac 100644
--- a/amazon-kinesis-client/pom.xml
+++ b/amazon-kinesis-client/pom.xml
@@ -22,7 +22,7 @@
   
     software.amazon.kinesis
     amazon-kinesis-client-pom
-    2.5.1-SNAPSHOT
+    2.5.1
   
 
   amazon-kinesis-client
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
index 5d01edeb..ea302b19 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
@@ -49,7 +49,7 @@ public class RetrievalConfig {
      */
     public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java";
 
-    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.1-SNAPSHOT";
+    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.1";
 
     /**
      * Client used to make calls to Kinesis for records retrieval
diff --git a/pom.xml b/pom.xml
index 90cec514..dcebcb9c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,7 +22,7 @@
   amazon-kinesis-client-pom
   pom
   Amazon Kinesis Client Library
-  2.5.1-SNAPSHOT
+  2.5.1
   The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
     from Amazon Kinesis.
   

From a9b0d00852208ef2fac587633c58d561c30e1672 Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Tue, 27 Jun 2023 15:24:03 -0400
Subject: [PATCH 76/88] Checkstyle: added additional checks to, primarily,
 safeguard against bugs. (#1154)

---
 .../multilang/messages/MessageTest.java        |  2 +-
 .../kinesis/retrieval/RetrievalConfig.java     |  2 +-
 ...rdShardRecordProcessorCheckpointerTest.java | 11 ++++-------
 .../metrics/EndingMetricsScopeTest.java        |  1 -
 .../retrieval/AWSExceptionManagerTest.java     | 18 ++++++------------
 .../retrieval/ThrottlingReporterTest.java      |  3 ---
 checkstyle/checkstyle.xml                      |  7 +++++++
 7 files changed, 19 insertions(+), 25 deletions(-)

diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java
index 86798080..62e5a741 100644
--- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java
+++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/messages/MessageTest.java
@@ -49,7 +49,7 @@ public class MessageTest {
                 new ProcessRecordsMessage(),
                 new ShutdownRequestedMessage(),
                 new LeaseLostMessage(),
-                new ShardEndedMessage()
+                new ShardEndedMessage(),
         };
 
 //        TODO: fix this
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
index ea302b19..3cb4a0d9 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
@@ -152,7 +152,7 @@ public class RetrievalConfig {
         if (streamTracker().isMultiStream()) {
             throw new IllegalArgumentException(
                     "Cannot set initialPositionInStreamExtended when multiStreamTracker is set");
-        };
+        }
 
         final StreamIdentifier streamIdentifier = getSingleStreamIdentifier();
         final StreamConfig updatedConfig = new StreamConfig(streamIdentifier, initialPositionInStreamExtended);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java
index 37a40b6b..98ce1dc5 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/ShardShardRecordProcessorCheckpointerTest.java
@@ -397,7 +397,7 @@ public class ShardShardRecordProcessorCheckpointerTest {
         assertThat(checkpointer.largestPermittedCheckpointValue(), equalTo(sequenceNumber));
     }
 
-    /*
+    /**
      * This test is a mixed test of checking some basic functionality of checkpointing at a sequence number and making
      * sure certain bounds checks and validations are being performed inside the checkpointer to prevent clients from
      * checkpointing out of order/too big/non-numeric values that aren't valid strings for them to be checkpointing
@@ -444,7 +444,7 @@ public class ShardShardRecordProcessorCheckpointerTest {
                 new ExtendedSequenceNumber("bogus-checkpoint-value"), // Can't checkpoint at non-numeric string
                 ExtendedSequenceNumber.SHARD_END, // Can't go to the end unless it is set as the max
                 ExtendedSequenceNumber.TRIM_HORIZON, // Can't go back to an initial sentinel value
-                ExtendedSequenceNumber.LATEST // Can't go back to an initial sentinel value
+                ExtendedSequenceNumber.LATEST, // Can't go back to an initial sentinel value
         };
         for (ExtendedSequenceNumber badCheckpointValue : valuesWeShouldNotBeAbleToCheckpointAt) {
             try {
@@ -477,7 +477,7 @@ public class ShardShardRecordProcessorCheckpointerTest {
                 processingCheckpointer.lastCheckpointValue(), equalTo(ExtendedSequenceNumber.SHARD_END));
     }
 
-    /*
+    /**
      * This test is a mixed test of checking some basic functionality of two phase checkpointing at a sequence number
      * and making sure certain bounds checks and validations are being performed inside the checkpointer to prevent
      * clients from checkpointing out of order/too big/non-numeric values that aren't valid strings for them to be
@@ -548,7 +548,7 @@ public class ShardShardRecordProcessorCheckpointerTest {
                         new ExtendedSequenceNumber("bogus-checkpoint-value"), // Can't checkpoint at non-numeric string
                         ExtendedSequenceNumber.SHARD_END, // Can't go to the end unless it is set as the max
                         ExtendedSequenceNumber.TRIM_HORIZON, // Can't go back to an initial sentinel value
-                        ExtendedSequenceNumber.LATEST // Can't go back to an initial sentinel value
+                        ExtendedSequenceNumber.LATEST, // Can't go back to an initial sentinel value
                 };
         for (ExtendedSequenceNumber badCheckpointValue : valuesWeShouldNotBeAbleToCheckpointAt) {
             try {
@@ -566,7 +566,6 @@ public class ShardShardRecordProcessorCheckpointerTest {
             assertThat("Largest sequence number should not have changed",
                     processingCheckpointer.largestPermittedCheckpointValue(), equalTo(thirdSequenceNumber));
             assertThat(checkpoint.getCheckpointObject(shardId).pendingCheckpoint(), nullValue());
-
         }
 
         // advance to third number
@@ -601,7 +600,6 @@ public class ShardShardRecordProcessorCheckpointerTest {
      *
      * @throws Exception
      */
-    @SuppressWarnings("serial")
     @Test
     public final void testMixedCheckpointCalls() throws Exception {
         for (LinkedHashMap testPlan : getMixedCallsTestPlan()) {
@@ -617,7 +615,6 @@ public class ShardShardRecordProcessorCheckpointerTest {
      *
      * @throws Exception
      */
-    @SuppressWarnings("serial")
     @Test
     public final void testMixedTwoPhaseCheckpointCalls() throws Exception {
         for (LinkedHashMap testPlan : getMixedCallsTestPlan()) {
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/EndingMetricsScopeTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/EndingMetricsScopeTest.java
index 2a32764d..a3d792ae 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/EndingMetricsScopeTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/metrics/EndingMetricsScopeTest.java
@@ -17,7 +17,6 @@ package software.amazon.kinesis.metrics;
 import org.junit.Test;
 
 import software.amazon.awssdk.services.cloudwatch.model.StandardUnit;
-import software.amazon.kinesis.metrics.EndingMetricsScope;
 
 public class EndingMetricsScopeTest {
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/AWSExceptionManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/AWSExceptionManagerTest.java
index 8319a0ac..030979df 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/AWSExceptionManagerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/AWSExceptionManagerTest.java
@@ -27,31 +27,28 @@ import static org.junit.Assert.assertThat;
 @Slf4j
 public class AWSExceptionManagerTest {
 
+    private static final String EXPECTED_HANDLING_MARKER = AWSExceptionManagerTest.class.getSimpleName();
+
+    private final AWSExceptionManager manager = new AWSExceptionManager();
+
     @Test
     public void testSpecificException() {
-        AWSExceptionManager manager = new AWSExceptionManager();
-        final String EXPECTED_HANDLING_MARKER = "Handled-TestException";
-
         manager.add(TestException.class, t -> {
             log.info("Handling test exception: {} -> {}", t.getMessage(), t.getAdditionalMessage());
             return new RuntimeException(EXPECTED_HANDLING_MARKER, t);
         });
 
-        TestException te = new TestException("Main Mesage", "Sub Message");
-
+        TestException te = new TestException("Main Message", "Sub Message");
 
         RuntimeException converted = manager.apply(te);
 
         assertThat(converted, isA(RuntimeException.class));
         assertThat(converted.getMessage(), equalTo(EXPECTED_HANDLING_MARKER));
         assertThat(converted.getCause(), equalTo(te));
-
     }
 
     @Test
     public void testParentException() {
-        AWSExceptionManager manager = new AWSExceptionManager();
-        final String EXPECTED_HANDLING_MARKER = "Handled-IllegalStateException";
         manager.add(IllegalArgumentException.class, i -> new RuntimeException("IllegalArgument", i));
         manager.add(Exception.class, i -> new RuntimeException("RawException", i));
         manager.add(IllegalStateException.class, i -> new RuntimeException(EXPECTED_HANDLING_MARKER, i));
@@ -66,8 +63,7 @@ public class AWSExceptionManagerTest {
 
     @Test
     public void testDefaultHandler() {
-        final String EXPECTED_HANDLING_MARKER = "Handled-Default";
-        AWSExceptionManager manager = new AWSExceptionManager().defaultFunction(i -> new RuntimeException(EXPECTED_HANDLING_MARKER, i));
+        manager.defaultFunction(i -> new RuntimeException(EXPECTED_HANDLING_MARKER, i));
 
         manager.add(IllegalArgumentException.class, i -> new RuntimeException("IllegalArgument", i));
         manager.add(Exception.class, i -> new RuntimeException("RawException", i));
@@ -83,8 +79,6 @@ public class AWSExceptionManagerTest {
 
     @Test
     public void testIdHandler() {
-        AWSExceptionManager manager = new AWSExceptionManager();
-
         manager.add(IllegalArgumentException.class, i -> new RuntimeException("IllegalArgument", i));
         manager.add(Exception.class, i -> new RuntimeException("RawException", i));
         manager.add(IllegalStateException.class, i -> i);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/ThrottlingReporterTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/ThrottlingReporterTest.java
index eec5ea9e..f13f0ad0 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/ThrottlingReporterTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/ThrottlingReporterTest.java
@@ -24,7 +24,6 @@ import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
 import org.slf4j.Logger;
-import software.amazon.kinesis.retrieval.ThrottlingReporter;
 
 @RunWith(MockitoJUnitRunner.class)
 public class ThrottlingReporterTest {
@@ -40,7 +39,6 @@ public class ThrottlingReporterTest {
         reporter.throttled();
         verify(throttleLog).warn(anyString());
         verify(throttleLog, never()).error(anyString());
-
     }
 
     @Test
@@ -63,7 +61,6 @@ public class ThrottlingReporterTest {
         reporter.throttled();
         verify(throttleLog, times(2)).warn(anyString());
         verify(throttleLog, times(3)).error(anyString());
-
     }
 
     private class LogTestingThrottingReporter extends ThrottlingReporter {
diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml
index 10031e23..76c4b330 100644
--- a/checkstyle/checkstyle.xml
+++ b/checkstyle/checkstyle.xml
@@ -23,8 +23,13 @@
 
     
         
+        
         
+        
+        
+        
         
+        
         
         
         
@@ -36,7 +41,9 @@
         
         
         
+        
         
+        
         
     
 

From feadd5e043522456e5bd9cdc0c551b01547e26ec Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Wed, 28 Jun 2023 10:36:32 -0400
Subject: [PATCH 77/88] Fix NPE on graceful shutdown before DDB
 `LeaseCoordinator` starts. (#1157)

---
 .../dynamodb/DynamoDBLeaseCoordinator.java    |  5 +--
 .../DynamoDBLeaseCoordinatorTest.java         | 32 ++++++++++++++-----
 2 files changed, 27 insertions(+), 10 deletions(-)

diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java
index 07e9068d..da6d8e07 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java
@@ -329,8 +329,9 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator {
 
     @Override
     public void stopLeaseTaker() {
-        takerFuture.cancel(false);
-
+        if (takerFuture != null) {
+            takerFuture.cancel(false);
+        }
     }
 
     @Override
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java
index caa7a6c7..cf1c536b 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorTest.java
@@ -1,6 +1,5 @@
 package software.amazon.kinesis.leases.dynamodb;
 
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -12,7 +11,7 @@ import software.amazon.kinesis.metrics.MetricsFactory;
 
 import java.util.UUID;
 
-import static org.mockito.Mockito.times;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
@@ -51,17 +50,34 @@ public class DynamoDBLeaseCoordinatorTest {
 
         leaseCoordinator.initialize();
 
-        verify(leaseRefresher, times(1)).createLeaseTableIfNotExists();
-        verify(leaseRefresher, times(1)).waitUntilLeaseTableExists(SECONDS_BETWEEN_POLLS, TIMEOUT_SECONDS);
+        verify(leaseRefresher).createLeaseTableIfNotExists();
+        verify(leaseRefresher).waitUntilLeaseTableExists(SECONDS_BETWEEN_POLLS, TIMEOUT_SECONDS);
     }
 
-    @Test
+    @Test(expected = DependencyException.class)
     public void testInitialize_tableCreationFails() throws Exception {
         when(leaseRefresher.createLeaseTableIfNotExists()).thenReturn(false);
         when(leaseRefresher.waitUntilLeaseTableExists(SECONDS_BETWEEN_POLLS, TIMEOUT_SECONDS)).thenReturn(false);
 
-        Assert.assertThrows(DependencyException.class, () -> leaseCoordinator.initialize());
-        verify(leaseRefresher, times(1)).createLeaseTableIfNotExists();
-        verify(leaseRefresher, times(1)).waitUntilLeaseTableExists(SECONDS_BETWEEN_POLLS, TIMEOUT_SECONDS);
+        try {
+            leaseCoordinator.initialize();
+        } finally {
+            verify(leaseRefresher).createLeaseTableIfNotExists();
+            verify(leaseRefresher).waitUntilLeaseTableExists(SECONDS_BETWEEN_POLLS, TIMEOUT_SECONDS);
+        }
     }
+
+    /**
+     * Validates a {@link NullPointerException} is not thrown when the lease taker
+     * is stopped before it starts/exists.
+     *
+     * @see issue #745
+     * @see issue #900
+     */
+    @Test
+    public void testStopLeaseTakerBeforeStart() {
+        leaseCoordinator.stopLeaseTaker();
+        assertTrue(leaseCoordinator.getAssignments().isEmpty());
+    }
+
 }

From 290facdd51ee0ecc530d36a165a13a07641cfb29 Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Wed, 28 Jun 2023 16:13:15 -0400
Subject: [PATCH 78/88] Modified `dependabot.yml` to set the correct `v[1|2].x`
 label. (#1151)

---
 .github/dependabot.yml | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/.github/dependabot.yml b/.github/dependabot.yml
index cad9988a..fd4483ad 100644
--- a/.github/dependabot.yml
+++ b/.github/dependabot.yml
@@ -2,12 +2,18 @@ version: 2
 updates:
 - package-ecosystem: "maven"
   directory: "/"
+  labels:
+    - "dependencies"
+    - "v2.x"
   schedule: 
     interval: "weekly"
 
 # branch - v1.x
 - package-ecosystem: "maven"
   directory: "/"
+  labels:
+    - "dependencies"
+    - "v1.x"
   target-branch: "v1.x"
   schedule:
     interval: "weekly"

From 42eb753d62a2ddb50cc7eb34a2a94fbb4986e6a7 Mon Sep 17 00:00:00 2001
From: Meher M <134537845+mmankika-aws@users.noreply.github.com>
Date: Thu, 29 Jun 2023 21:51:07 -0700
Subject: [PATCH 79/88] Bug fix in lease refresher integration test with
 occasional failures (#1159)

---
 .../DynamoDBLeaseRefresherIntegrationTest.java  | 17 ++++++-----------
 1 file changed, 6 insertions(+), 11 deletions(-)

diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java
index 1b2fa78a..51e12d2d 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java
@@ -17,6 +17,7 @@ package software.amazon.kinesis.leases.dynamodb;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.junit.Assert;
 import org.junit.Before;
@@ -34,7 +35,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.verify;
@@ -299,17 +299,12 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest
 
     @Test
     public void testWaitUntilLeaseTableExists() throws LeasingException {
-        DynamoDBLeaseRefresher refresher = new DynamoDBLeaseRefresher("nagl_ShardProgress", ddbClient,
-                new DynamoDBLeaseSerializer(), true, tableCreatorCallback) {
-            @Override
-            long sleep(long timeToSleepMillis) {
-                fail("Should not sleep");
-                return 0L;
-            }
+        final UUID uniqueId = UUID.randomUUID();
+        DynamoDBLeaseRefresher refresher = new DynamoDBLeaseRefresher("tableEventuallyExists_" + uniqueId, ddbClient,
+                new DynamoDBLeaseSerializer(), true, tableCreatorCallback);
 
-        };
-
-        assertTrue(refresher.waitUntilLeaseTableExists(1, 1));
+        refresher.createLeaseTableIfNotExists();
+        assertTrue(refresher.waitUntilLeaseTableExists(1, 20));
     }
 
     @Test

From 8d1ee6b5e16ecb53c393b24439b8fc566fa29de0 Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Thu, 6 Jul 2023 13:24:14 -0400
Subject: [PATCH 80/88] Checkstyle: tightened `LineLength` restriction from 170
 to 150. (#1158)

---
 .../credentials/V2CredentialWrapper.java      |   3 +-
 ...tialsProviderPropertyValueDecoderTest.java |   4 +-
 .../KinesisClientLibConfiguratorTest.java     | 104 ++++--------------
 .../amazon/kinesis/coordinator/Scheduler.java |  17 ++-
 .../leases/HierarchicalShardSyncer.java       |  18 ++-
 .../kinesis/leases/LeaseCleanupManager.java   |  13 ++-
 .../dynamodb/DynamoDBLeaseRefresher.java      |   3 +-
 .../dynamodb/DynamoDBLeaseSerializer.java     |   9 +-
 .../kinesis/lifecycle/ShutdownTask.java       |   9 +-
 .../fanout/FanOutRecordsPublisher.java        |  21 +++-
 .../kinesis/coordinator/SchedulerTest.java    |  33 +++---
 .../ExceptionThrowingLeaseRefresher.java      |   3 +-
 .../leases/HierarchicalShardSyncerTest.java   |  18 ++-
 .../kinesis/leases/ShardObjectHelper.java     |   3 +-
 .../DynamoDBLeaseTakerIntegrationTest.java    |   3 +-
 .../kinesis/lifecycle/ConsumerStatesTest.java |  11 +-
 .../kinesis/lifecycle/ProcessTaskTest.java    |  17 ++-
 .../fanout/FanOutRecordsPublisherTest.java    |  19 ++--
 ...ronousGetRecordsRetrievalStrategyTest.java |   6 +-
 ...efetchRecordsPublisherIntegrationTest.java |   3 +-
 .../polling/PrefetchRecordsPublisherTest.java |   3 +-
 checkstyle/checkstyle.xml                     |   4 +-
 22 files changed, 157 insertions(+), 167 deletions(-)

diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/credentials/V2CredentialWrapper.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/credentials/V2CredentialWrapper.java
index d0afdf75..50880a83 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/credentials/V2CredentialWrapper.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/credentials/V2CredentialWrapper.java
@@ -32,7 +32,8 @@ public class V2CredentialWrapper implements AwsCredentialsProvider {
     public AwsCredentials resolveCredentials() {
         AWSCredentials current = oldCredentialsProvider.getCredentials();
         if (current instanceof AWSSessionCredentials) {
-            return AwsSessionCredentials.create(current.getAWSAccessKeyId(), current.getAWSSecretKey(), ((AWSSessionCredentials) current).getSessionToken());
+            return AwsSessionCredentials.create(current.getAWSAccessKeyId(), current.getAWSSecretKey(),
+                    ((AWSSessionCredentials) current).getSessionToken());
         }
         return new AwsCredentials() {
             @Override
diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java
index 36f496d3..ced63f24 100644
--- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java
+++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java
@@ -36,8 +36,8 @@ public class AWSCredentialsProviderPropertyValueDecoderTest {
     private static final String TEST_ACCESS_KEY_ID = "123";
     private static final String TEST_SECRET_KEY = "456";
 
-    private String credentialName1 = "software.amazon.kinesis.multilang.config.AWSCredentialsProviderPropertyValueDecoderTest$AlwaysSucceedCredentialsProvider";
-    private String credentialName2 = "software.amazon.kinesis.multilang.config.AWSCredentialsProviderPropertyValueDecoderTest$ConstructorCredentialsProvider";
+    private final String credentialName1 = AlwaysSucceedCredentialsProvider.class.getName();
+    private final String credentialName2 = ConstructorCredentialsProvider.class.getName();
     private AWSCredentialsProviderPropertyValueDecoder decoder = new AWSCredentialsProviderPropertyValueDecoder();
 
     @ToString
diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java
index 1f05240a..2b02ea43 100644
--- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java
+++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfiguratorTest.java
@@ -36,35 +36,25 @@ import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.exception.ExceptionUtils;
-import org.junit.Rule;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
 
-import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
-import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
 import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.kinesis.common.InitialPositionInStream;
 import software.amazon.kinesis.metrics.MetricsLevel;
-import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
 
 @RunWith(MockitoJUnitRunner.class)
 public class KinesisClientLibConfiguratorTest {
 
-    private String credentialName1 = "software.amazon.kinesis.multilang.config.KinesisClientLibConfiguratorTest$AlwaysSucceedCredentialsProvider";
-    private String credentialName2 = "software.amazon.kinesis.multilang.config.KinesisClientLibConfiguratorTest$AlwaysFailCredentialsProvider";
-    private String credentialNameKinesis = "software.amazon.kinesis.multilang.config.KinesisClientLibConfiguratorTest$AlwaysSucceedCredentialsProviderKinesis";
-    private String credentialNameDynamoDB = "software.amazon.kinesis.multilang.config.KinesisClientLibConfiguratorTest$AlwaysSucceedCredentialsProviderDynamoDB";
-    private String credentialNameCloudWatch = "software.amazon.kinesis.multilang.config.KinesisClientLibConfiguratorTest$AlwaysSucceedCredentialsProviderCloudWatch";
-    private KinesisClientLibConfigurator configurator = new KinesisClientLibConfigurator();
-
-    @Rule
-    public final ExpectedException thrown = ExpectedException.none();
-
-    @Mock
-    private ShardRecordProcessorFactory shardRecordProcessorFactory;
+    private final String credentialName1 = AlwaysSucceedCredentialsProvider.class.getName();
+    private final String credentialName2 = AlwaysFailCredentialsProvider.class.getName();
+    private final String credentialNameKinesis = AlwaysSucceedCredentialsProviderKinesis.class.getName();
+    private final String credentialNameDynamoDB = AlwaysSucceedCredentialsProviderDynamoDB.class.getName();
+    private final String credentialNameCloudWatch = AlwaysSucceedCredentialsProviderCloudWatch.class.getName();
+    private final KinesisClientLibConfigurator configurator = new KinesisClientLibConfigurator();
 
     @Test
     public void testWithBasicSetup() {
@@ -241,52 +231,32 @@ public class KinesisClientLibConfiguratorTest {
                 "AWSCredentialsProvider = ABCD," + credentialName1, "workerId = 123",
                 "initialPositionInStream = TriM_Horizon", "maxGetRecordsThreadPool = 0",
                 "retryGetRecordsInSeconds = 0" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
-
-        try {
-            configurator.getConfiguration(input);
-        } catch (Exception e) {
-            fail("Don't expect to fail on invalid variable value");
-
-        }
+        getConfiguration(test);
     }
 
     @Test
     public void testWithInvalidIntValue() {
         String test = StringUtils.join(new String[] { "streamName = a", "applicationName = b",
                 "AWSCredentialsProvider = " + credentialName1, "workerId = 123", "failoverTimeMillis = 100nf" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
-
-        try {
-            configurator.getConfiguration(input);
-        } catch (Exception e) {
-            fail("Don't expect to fail on invalid variable value");
-        }
+        getConfiguration(test);
     }
 
     @Test
     public void testWithNegativeIntValue() {
         String test = StringUtils.join(new String[] { "streamName = a", "applicationName = b",
                 "AWSCredentialsProvider = " + credentialName1, "workerId = 123", "failoverTimeMillis = -12" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
 
         // separate input stream with getConfiguration to explicitly catch exception from the getConfiguration statement
-        try {
-            configurator.getConfiguration(input);
-        } catch (Exception e) {
-            fail("Don't expect to fail on invalid variable value");
-        }
+        getConfiguration(test);
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void testWithMissingCredentialsProvider() {
-
         String test = StringUtils.join(new String[] { "streamName = a", "applicationName = b", "workerId = 123",
                 "failoverTimeMillis = 100", "shardSyncIntervalMillis = 500" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
 
         // separate input stream with getConfiguration to explicitly catch exception from the getConfiguration statement
-        configurator.getConfiguration(input);
+        getConfiguration(test);
     }
 
     @Test
@@ -295,8 +265,7 @@ public class KinesisClientLibConfiguratorTest {
                 new String[] { "streamName = a", "applicationName = b", "AWSCredentialsProvider = " + credentialName1,
                         "failoverTimeMillis = 100", "shardSyncIntervalMillis = 500" },
                 '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
-        MultiLangDaemonConfiguration config = configurator.getConfiguration(input);
+        MultiLangDaemonConfiguration config = getConfiguration(test);
 
         // if workerId is not provided, configurator should assign one for it automatically
         assertNotNull(config.getWorkerIdentifier());
@@ -311,14 +280,11 @@ public class KinesisClientLibConfiguratorTest {
                         "workerId = 123",
                         "failoverTimeMillis = 100" },
                 '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
-
-        configurator.getConfiguration(input);
+        getConfiguration(test);
     }
 
     @Test(expected = IllegalArgumentException.class)
     public void testWithEmptyStreamNameAndMissingStreamArn() {
-
         String test = StringUtils.join(new String[] {
                         "applicationName = b",
                         "AWSCredentialsProvider = " + credentialName1,
@@ -327,18 +293,14 @@ public class KinesisClientLibConfiguratorTest {
                         "streamName = ",
                         "streamArn = "},
                 '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
-
-        configurator.getConfiguration(input);
+        getConfiguration(test);
     }
 
     @Test(expected = NullPointerException.class)
     public void testWithMissingApplicationName() {
-
         String test = StringUtils.join(new String[] { "streamName = a", "AWSCredentialsProvider = " + credentialName1,
                 "workerId = 123", "failoverTimeMillis = 100" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
-        configurator.getConfiguration(input);
+        getConfiguration(test);
     }
 
     @Test
@@ -347,11 +309,10 @@ public class KinesisClientLibConfiguratorTest {
                 new String[] { "streamName = a", "applicationName = b", "AWSCredentialsProvider = " + credentialName2,
                         "failoverTimeMillis = 100", "shardSyncIntervalMillis = 500" },
                 '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
+        MultiLangDaemonConfiguration config = getConfiguration(test);
 
         // separate input stream with getConfiguration to explicitly catch exception from the getConfiguration statement
         try {
-            MultiLangDaemonConfiguration config = configurator.getConfiguration(input);
             config.getKinesisCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
             fail("expect failure with wrong credentials provider");
         } catch (Exception e) {
@@ -367,25 +328,12 @@ public class KinesisClientLibConfiguratorTest {
                 "AWSCredentialsProviderDynamoDB = " + credentialNameDynamoDB,
                 "AWSCredentialsProviderCloudWatch = " + credentialNameCloudWatch, "failoverTimeMillis = 100",
                 "shardSyncIntervalMillis = 500" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
 
         // separate input stream with getConfiguration to explicitly catch exception from the getConfiguration statement
-        MultiLangDaemonConfiguration config = configurator.getConfiguration(input);
-        try {
-            config.getKinesisCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
-        } catch (Exception e) {
-            fail("Kinesis credential providers should not fail.");
-        }
-        try {
-            config.getDynamoDBCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
-        } catch (Exception e) {
-            fail("DynamoDB credential providers should not fail.");
-        }
-        try {
-            config.getCloudWatchCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
-        } catch (Exception e) {
-            fail("CloudWatch credential providers should not fail.");
-        }
+        final MultiLangDaemonConfiguration config = getConfiguration(test);
+        config.getKinesisCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
+        config.getDynamoDBCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
+        config.getCloudWatchCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
     }
 
     // TODO: fix this test
@@ -396,17 +344,10 @@ public class KinesisClientLibConfiguratorTest {
                 "AWSCredentialsProviderDynamoDB = " + credentialName2,
                 "AWSCredentialsProviderCloudWatch = " + credentialName2, "failoverTimeMillis = 100",
                 "shardSyncIntervalMillis = 500" }, '\n');
-        InputStream input = new ByteArrayInputStream(test.getBytes());
 
         // separate input stream with getConfiguration to explicitly catch exception from the getConfiguration statement
-
-        // separate input stream with getConfiguration to explicitly catch exception from the getConfiguration statement
-        MultiLangDaemonConfiguration config = configurator.getConfiguration(input);
-        try {
-            config.getKinesisCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
-        } catch (Exception e) {
-            fail("Kinesis credential providers should not fail.");
-        }
+        final MultiLangDaemonConfiguration config = getConfiguration(test);
+        config.getKinesisCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
         try {
             config.getDynamoDBCredentialsProvider().build(AwsCredentialsProvider.class).resolveCredentials();
             fail("DynamoDB credential providers should fail.");
@@ -503,7 +444,6 @@ public class KinesisClientLibConfiguratorTest {
 
     private MultiLangDaemonConfiguration getConfiguration(String configString) {
         InputStream input = new ByteArrayInputStream(configString.getBytes());
-        MultiLangDaemonConfiguration config = configurator.getConfiguration(input);
-        return config;
+        return configurator.getConfiguration(input);
     }
 }
\ No newline at end of file
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java
index d67fff96..11983ded 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java
@@ -97,6 +97,7 @@ import software.amazon.kinesis.retrieval.RetrievalConfig;
 import software.amazon.kinesis.schemaregistry.SchemaRegistryDecoder;
 
 import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.StreamsLeasesDeletionType;
+import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION;
 
 /**
  *
@@ -489,7 +490,7 @@ public class Scheduler implements Runnable {
                     }
                 };
 
-                if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) {
+                if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) {
                     // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion.
                     // It is assumed that all the workers will always have the latest and consistent snapshot of streams
                     // from the multiStreamTracker.
@@ -521,7 +522,8 @@ public class Scheduler implements Runnable {
                         if (!newStreamConfigMap.containsKey(streamIdentifier)) {
                             if (SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS) {
                                 log.info(
-                                        "Found old/deleted stream : {}. Triggering shard sync. Removing from tracked active streams.", streamIdentifier);
+                                        "Found old/deleted stream : {}. Triggering shard sync. Removing from tracked active streams.",
+                                        streamIdentifier);
                                 ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(
                                         currentStreamConfigMap.get(streamIdentifier));
                                 shardSyncTaskManager.submitShardSyncTask();
@@ -541,10 +543,13 @@ public class Scheduler implements Runnable {
                 // Now let's scan the streamIdentifiersForLeaseCleanup eligible for deferred deletion and delete them.
                 // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and
                 // the streamIdentifiersForLeaseCleanup are not present in the latest snapshot.
-                final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors
-                        .partitioningBy(newStreamConfigMap::containsKey, Collectors.toSet()));
-                final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier ->
-                        Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis())
+                final Map> staleStreamIdDeletionDecisionMap =
+                        staleStreamDeletionMap.keySet().stream().collect(
+                                Collectors.partitioningBy(newStreamConfigMap::containsKey, Collectors.toSet()));
+                final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false)
+                        .stream().filter(streamIdentifier ->
+                                Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now())
+                                        .toMillis() >= waitPeriodToDeleteOldStreams.toMillis())
                         .collect(Collectors.toSet());
                 // These are the streams which are deleted in Kinesis and we encounter resource not found during
                 // shardSyncTask. This is applicable in MultiStreamMode only, in case of SingleStreamMode, store will
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java
index 534b5fd3..11014505 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java
@@ -92,7 +92,8 @@ public class HierarchicalShardSyncer {
         this(isMultiStreamMode, streamIdentifier, null);
     }
 
-    public HierarchicalShardSyncer(final boolean isMultiStreamMode, final String streamIdentifier, final DeletedStreamListProvider deletedStreamListProvider) {
+    public HierarchicalShardSyncer(final boolean isMultiStreamMode, final String streamIdentifier,
+            final DeletedStreamListProvider deletedStreamListProvider) {
         this.isMultiStreamMode = isMultiStreamMode;
         this.streamIdentifier = streamIdentifier;
         this.deletedStreamListProvider = deletedStreamListProvider;
@@ -191,7 +192,9 @@ public class HierarchicalShardSyncer {
         if (!CollectionUtils.isNullOrEmpty(inconsistentShardIds)) {
             final String ids = StringUtils.join(inconsistentShardIds, ' ');
             throw new KinesisClientLibIOException(String.format(
+                    // CHECKSTYLE.OFF: LineLength
                     "%d open child shards (%s) are inconsistent. This can happen due to a race condition between describeStream and a reshard operation.",
+                    // CHECKSTYLE.ON: LineLength
                     inconsistentShardIds.size(), ids));
         }
     }
@@ -564,7 +567,8 @@ public class HierarchicalShardSyncer {
         return parentShardIds;
     }
 
-    public synchronized Lease createLeaseForChildShard(final ChildShard childShard, final StreamIdentifier streamIdentifier) throws InvalidStateException {
+    public synchronized Lease createLeaseForChildShard(final ChildShard childShard,
+            final StreamIdentifier streamIdentifier) throws InvalidStateException {
         final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, streamIdentifier);
 
         return multiStreamArgs.isMultiStreamMode() ? newKCLMultiStreamLeaseForChildShard(childShard, streamIdentifier)
@@ -583,7 +587,8 @@ public class HierarchicalShardSyncer {
         if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) {
             newLease.parentShardIds(childShard.parentShards());
         } else {
-            throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found.");
+            throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId()
+                    + " because parent shards cannot be found.");
         }
         newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
         newLease.ownerSwitchesSinceCheckpoint(0L);
@@ -591,13 +596,15 @@ public class HierarchicalShardSyncer {
         return newLease;
     }
 
-    private static Lease newKCLMultiStreamLeaseForChildShard(final ChildShard childShard, final StreamIdentifier streamIdentifier) throws InvalidStateException {
+    private static Lease newKCLMultiStreamLeaseForChildShard(final ChildShard childShard,
+            final StreamIdentifier streamIdentifier) throws InvalidStateException {
         MultiStreamLease newLease = new MultiStreamLease();
         newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), childShard.shardId()));
         if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) {
             newLease.parentShardIds(childShard.parentShards());
         } else {
-            throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found.");
+            throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId()
+                    + " because parent shards cannot be found.");
         }
         newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON);
         newLease.ownerSwitchesSinceCheckpoint(0L);
@@ -612,7 +619,6 @@ public class HierarchicalShardSyncer {
      * Note: Package level access only for testing purposes
      *
      * @param shard
-     * @return
      */
     private static Lease newKCLLease(final Shard shard) {
         Lease newLease = new Lease();
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java
index 8a442bd3..861626b6 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java
@@ -208,7 +208,8 @@ public class LeaseCleanupManager {
                         log.warn("Unable to cleanup lease for shard {} in {}", shardInfo.shardId(), streamIdentifier.streamName(), e);
                     }
                 } else {
-                    log.info("Lease not present in lease table while cleaning the shard {} of {}", shardInfo.shardId(), streamIdentifier.streamName());
+                    log.info("Lease not present in lease table while cleaning the shard {} of {}",
+                            shardInfo.shardId(), streamIdentifier.streamName());
                     cleanedUpCompletedLease = true;
                 }
             }
@@ -232,14 +233,17 @@ public class LeaseCleanupManager {
 
     // A lease that ended with SHARD_END from ResourceNotFoundException is safe to delete if it no longer exists in the
     // stream (known explicitly from ResourceNotFound being thrown when processing this shard),
-    private boolean cleanupLeaseForGarbageShard(Lease lease, Throwable e) throws DependencyException, ProvisionedThroughputException, InvalidStateException {
+    private boolean cleanupLeaseForGarbageShard(Lease lease, Throwable e)
+            throws DependencyException, ProvisionedThroughputException, InvalidStateException {
         log.warn("Deleting lease {} as it is not present in the stream.", lease, e);
         leaseCoordinator.leaseRefresher().deleteLease(lease);
         return true;
     }
 
     /**
-     * Check if the all of the parent shards for a given lease have an ongoing lease. If any one parent still has a lease, return false. Otherwise return true
+     * Check if the all of the parent shards for a given lease have an ongoing lease. If any one parent still has a
+     * lease, return false. Otherwise return true
+     *
      * @param lease
      * @param shardInfo
      * @return
@@ -247,7 +251,8 @@ public class LeaseCleanupManager {
      * @throws ProvisionedThroughputException
      * @throws InvalidStateException
      */
-    private boolean allParentShardLeasesDeleted(Lease lease, ShardInfo shardInfo) throws DependencyException, ProvisionedThroughputException, InvalidStateException {
+    private boolean allParentShardLeasesDeleted(Lease lease, ShardInfo shardInfo)
+            throws DependencyException, ProvisionedThroughputException, InvalidStateException {
         for (String parentShard : lease.parentShardIds()) {
             final Lease parentLease = leaseCoordinator.leaseRefresher().getLease(ShardInfo.getLeaseKey(shardInfo, parentShard));
 
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java
index 4bef8442..c87f3eb8 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java
@@ -152,7 +152,8 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher {
                                   final LeaseSerializer serializer, final boolean consistentReads,
                                   @NonNull final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout,
                                   final BillingMode billingMode) {
-        this(table, dynamoDBClient, serializer, consistentReads, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, DefaultSdkAutoConstructList.getInstance());
+        this(table, dynamoDBClient, serializer, consistentReads, tableCreatorCallback, dynamoDbRequestTimeout,
+                billingMode, DefaultSdkAutoConstructList.getInstance());
     }
 
     /**
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java
index 9ebed654..7e12b9a9 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java
@@ -224,11 +224,13 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
     public Map getDynamoTakeLeaseUpdate(final Lease lease, String owner) {
         Map result = new HashMap<>();
 
-        result.put(LEASE_OWNER_KEY, AttributeValueUpdate.builder().value(DynamoUtils.createAttributeValue(owner)).action(AttributeAction.PUT).build());
+        result.put(LEASE_OWNER_KEY, AttributeValueUpdate.builder().value(DynamoUtils.createAttributeValue(owner))
+                .action(AttributeAction.PUT).build());
 
         String oldOwner = lease.leaseOwner();
         if (oldOwner != null && !oldOwner.equals(owner)) {
-            result.put(OWNER_SWITCHES_KEY, AttributeValueUpdate.builder().value(DynamoUtils.createAttributeValue(1L)).action(AttributeAction.ADD).build());
+            result.put(OWNER_SWITCHES_KEY, AttributeValueUpdate.builder().value(DynamoUtils.createAttributeValue(1L))
+                    .action(AttributeAction.ADD).build());
         }
 
         return result;
@@ -257,7 +259,8 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer {
 
         if (lease.pendingCheckpoint() != null && !lease.pendingCheckpoint().sequenceNumber().isEmpty()) {
             result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.pendingCheckpoint().sequenceNumber())));
-            result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.pendingCheckpoint().subSequenceNumber())));
+            result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, putUpdate(DynamoUtils.createAttributeValue(
+                    lease.pendingCheckpoint().subSequenceNumber())));
         } else {
             result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
             result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build());
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java
index 4a96d87d..16398963 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java
@@ -180,7 +180,8 @@ public class ShutdownTask implements ConsumerTask {
         // Create new lease for the child shards if they don't exist.
         // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards.
         // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception.
-        // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards.
+        // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a
+        // shutdown task with empty list of childShards.
         // This scenario could happen when customer deletes the stream while leaving the KCL application running.
         if (currentShardLease == null) {
             throw new InvalidStateException(leaseKey
@@ -286,7 +287,8 @@ public class ShutdownTask implements ConsumerTask {
         for (ChildShard childShard : childShards) {
             final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId());
             if (leaseRefresher.getLease(leaseKey) == null) {
-                log.debug("{} - Shard {} - Attempting to create lease for child shard {}", shardDetector.streamIdentifier(), shardInfo.shardId(), leaseKey);
+                log.debug("{} - Shard {} - Attempting to create lease for child shard {}",
+                        shardDetector.streamIdentifier(), shardInfo.shardId(), leaseKey);
                 final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier());
                 final long startTime = System.currentTimeMillis();
                 boolean success = false;
@@ -296,7 +298,8 @@ public class ShutdownTask implements ConsumerTask {
                 } finally {
                     MetricsUtil.addSuccessAndLatency(scope, "CreateLease", success, startTime, MetricsLevel.DETAILED);
                     if (leaseToCreate.checkpoint() != null) {
-                        final String metricName = leaseToCreate.checkpoint().isSentinelCheckpoint() ? leaseToCreate.checkpoint().sequenceNumber() : "SEQUENCE_NUMBER";
+                        final String metricName = leaseToCreate.checkpoint().isSentinelCheckpoint() ?
+                                leaseToCreate.checkpoint().sequenceNumber() : "SEQUENCE_NUMBER";
                         MetricsUtil.addSuccess(scope, "CreateLease_" + metricName, true, MetricsLevel.DETAILED);
                     }
                 }
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java
index 7177211f..a17e5e82 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java
@@ -230,8 +230,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
                 subscriber.onNext(recordsRetrieved);
             }
         } catch (IllegalStateException e) {
-
+            // CHECKSTYLE.OFF: LineLength
             log.warn("{}: Unable to enqueue the payload due to capacity restrictions in delivery queue with remaining capacity {}. Last successful request details -- {}",
+                    // CHECKSTYLE.ON: LineLength
                     streamAndShardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails);
             throw e;
         } catch (Throwable t) {
@@ -382,7 +383,8 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
         // Clear any lingering records in the queue.
         if (!recordsDeliveryQueue.isEmpty()) {
             log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of "
-                    + "previous subscription - {}. Last successful request details -- {}", streamAndShardId, subscribeToShardId, lastSuccessfulRequestDetails);
+                    + "previous subscription - {}. Last successful request details -- {}",
+                    streamAndShardId, subscribeToShardId, lastSuccessfulRequestDetails);
             recordsDeliveryQueue.clear();
         }
     }
@@ -402,7 +404,8 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
             // The ack received for this onNext event will be ignored by the publisher as the global flow object should
             // be either null or renewed when the ack's flow identifier is evaluated.
             FanoutRecordsRetrieved response = new FanoutRecordsRetrieved(
-                    ProcessRecordsInput.builder().records(Collections.emptyList()).isAtShardEnd(true).childShards(Collections.emptyList()).build(), null,
+                    ProcessRecordsInput.builder().records(Collections.emptyList()).isAtShardEnd(true)
+                            .childShards(Collections.emptyList()).build(), null,
                     triggeringFlow != null ? triggeringFlow.getSubscribeToShardId() : shardId + "-no-flow-found");
             subscriber.onNext(response);
             subscriber.onComplete();
@@ -515,7 +518,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
     private void updateAvailableQueueSpaceAndRequestUpstream(RecordFlow triggeringFlow) {
         if (availableQueueSpace <= 0) {
             log.debug(
+                    // CHECKSTYLE.OFF: LineLength
                     "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Attempted to decrement availableQueueSpace to below 0",
+                    // CHECKSTYLE.ON: LineLength
                     streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
         } else {
             availableQueueSpace--;
@@ -544,7 +549,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
 
             if (!isActiveFlow(triggeringFlow)) {
                 log.debug(
+                        // CHECKSTYLE.OFF: LineLength
                         "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {} -- Received spurious onComplete from unexpected flow. Ignoring.",
+                        // CHECKSTYLE.ON: LineLength
                         streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId);
                 return;
             }
@@ -801,7 +808,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
                         throwable.getMessage());
                 if (this.isDisposed) {
                     log.debug(
+                            // CHECKSTYLE.OFF: LineLength
                             "{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- This flow has been disposed, not dispatching error. {}: {}",
+                            // CHECKSTYLE.ON: LineLength
                             parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(),
                             throwable.getMessage());
                     this.isErrorDispatched = true;
@@ -891,7 +900,9 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
                         subscription.cancel();
                     } catch (Throwable t) {
                         log.error(
+                                // CHECKSTYLE.OFF: LineLength
                                 "{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Exception while trying to cancel failed subscription: {}",
+                                // CHECKSTYLE.ON: LineLength
                                 parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t);
                     }
                 }
@@ -953,12 +964,16 @@ public class FanOutRecordsPublisher implements RecordsPublisher {
                 if (flow.shouldSubscriptionCancel()) {
                     if (flow.isCancelled) {
                         log.debug(
+                                // CHECKSTYLE.OFF: LineLength
                                 "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Subscription was cancelled before onSubscribe",
+                                // CHECKSTYLE.ON: LineLength
                                 parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
                     }
                     if (flow.isDisposed) {
                         log.debug(
+                                // CHECKSTYLE.OFF: LineLength
                                 "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow has been disposed cancelling subscribe",
+                                // CHECKSTYLE.ON: LineLength
                                 parent.streamAndShardId, connectionStartedAt, subscribeToShardId);
                     }
                     log.debug(
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java
index 3b5bfec9..fd1380f2 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java
@@ -38,6 +38,7 @@ import static org.mockito.internal.verification.VerificationModeFactory.atMost;
 
 import java.time.Duration;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
@@ -176,7 +177,8 @@ public class SchedulerTest {
         shardRecordProcessorFactory = new TestShardRecordProcessorFactory();
 
         checkpointConfig = new CheckpointConfig().checkpointFactory(new TestKinesisCheckpointFactory());
-        coordinatorConfig = new CoordinatorConfig(applicationName).parentShardPollIntervalMillis(100L).workerStateChangeListener(workerStateChangeListener);
+        coordinatorConfig = new CoordinatorConfig(applicationName).parentShardPollIntervalMillis(100L)
+                .workerStateChangeListener(workerStateChangeListener);
         leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName,
                 workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, false));
         lifecycleConfig = new LifecycleConfig();
@@ -188,7 +190,8 @@ public class SchedulerTest {
         when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector);
         when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(new HierarchicalShardSyncer());
         when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null));
-        when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(StreamConfig.class), any(MetricsFactory.class))).thenReturn(recordsPublisher);
+        when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(StreamConfig.class),
+                any(MetricsFactory.class))).thenReturn(recordsPublisher);
         when(shardDetector.streamIdentifier()).thenReturn(mock(StreamIdentifier.class));
 
         scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig,
@@ -645,7 +648,8 @@ public class SchedulerTest {
         testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null);
     }
 
-    private final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(boolean expectSyncedStreams, Set currentStreamConfigMapOverride)
+    private void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(boolean expectSyncedStreams,
+            Set currentStreamConfigMapOverride)
             throws DependencyException, ProvisionedThroughputException, InvalidStateException {
         List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig(
                 StreamIdentifier.multiStreamInstance(
@@ -1294,21 +1298,20 @@ public class SchedulerTest {
         }
     }
 
-    // TODO: Upgrade to mockito >= 2.7.13, and use Spy on MultiStreamTracker to directly access the default methods without implementing TestMultiStreamTracker class
+    // TODO: Upgrade to mockito >= 2.7.13, and use Spy on MultiStreamTracker to directly access the default methods
+    //  without implementing TestMultiStreamTracker class
     @NoArgsConstructor
     private class TestMultiStreamTracker implements MultiStreamTracker {
         @Override
-        public List streamConfigList(){
-            return new ArrayList() {{
-                add(new StreamConfig(StreamIdentifier.multiStreamInstance("123456789012:stream1:1"), InitialPositionInStreamExtended.newInitialPosition(
-                        InitialPositionInStream.LATEST)));
-                add(new StreamConfig(StreamIdentifier.multiStreamInstance("123456789012:stream2:2"), InitialPositionInStreamExtended.newInitialPosition(
-                        InitialPositionInStream.LATEST)));
-                add(new StreamConfig(StreamIdentifier.multiStreamInstance("210987654321:stream1:1"), InitialPositionInStreamExtended.newInitialPosition(
-                        InitialPositionInStream.LATEST)));
-                add(new StreamConfig(StreamIdentifier.multiStreamInstance("210987654321:stream2:3"), InitialPositionInStreamExtended.newInitialPosition(
-                        InitialPositionInStream.LATEST)));
-            }};
+        public List streamConfigList() {
+            final InitialPositionInStreamExtended latest =
+                    InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST);
+
+            return Arrays.asList(
+                    new StreamConfig(StreamIdentifier.multiStreamInstance("123456789012:stream1:1"), latest),
+                    new StreamConfig(StreamIdentifier.multiStreamInstance("123456789012:stream2:2"), latest),
+                    new StreamConfig(StreamIdentifier.multiStreamInstance("210987654321:stream1:1"), latest),
+                    new StreamConfig(StreamIdentifier.multiStreamInstance("210987654321:stream2:3"), latest));
         }
 
         @Override
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java
index 5e612ade..7f93216d 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java
@@ -141,7 +141,8 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher {
     }
 
     @Override
-    public List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException {
+    public List listLeasesForStream(StreamIdentifier streamIdentifier)
+            throws DependencyException, InvalidStateException, ProvisionedThroughputException {
         throwExceptions("listLeasesForStream", ExceptionThrowingLeaseRefresherMethods.LISTLEASESFORSTREAM);
 
         return leaseRefresher.listLeasesForStream(streamIdentifier);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java
index 1a1abc0e..6df34633 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java
@@ -2257,10 +2257,12 @@ public class HierarchicalShardSyncerTest {
         testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_AT_TIMESTAMP, shardFilter);
     }
 
-    public void testEmptyLeaseTableBootstrapUsesListShardsWithFilter(InitialPositionInStreamExtended initialPosition, ShardFilter shardFilter) throws Exception {
+    public void testEmptyLeaseTableBootstrapUsesListShardsWithFilter(InitialPositionInStreamExtended initialPosition,
+            ShardFilter shardFilter) throws Exception {
         final String shardId0 = "shardId-0";
         final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null,
-                ShardObjectHelper.newSequenceNumberRange("1", null), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, ShardObjectHelper.MAX_HASH_KEY)));
+                ShardObjectHelper.newSequenceNumberRange("1", null),
+                ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, ShardObjectHelper.MAX_HASH_KEY)));
 
         when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true);
         when(shardDetector.listShardsWithFilter(shardFilter)).thenReturn(shards);
@@ -2278,8 +2280,10 @@ public class HierarchicalShardSyncerTest {
         final String shardId0 = "shardId-0";
         final String shardId1 = "shardId-1";
 
-        final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, ShardObjectHelper.newSequenceNumberRange("1", "2")));
-        final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId1, null, null, ShardObjectHelper.newSequenceNumberRange("3", "4")));
+        final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null,
+                ShardObjectHelper.newSequenceNumberRange("1", "2")));
+        final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId1, null, null,
+                ShardObjectHelper.newSequenceNumberRange("3", "4")));
 
         final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo");
 
@@ -2301,8 +2305,10 @@ public class HierarchicalShardSyncerTest {
     @Test(expected = KinesisClientLibIOException.class)
     public void testEmptyLeaseTableThrowsExceptionWhenHashRangeIsStillIncompleteAfterRetries() throws Exception {
         final List shardsWithIncompleteHashRange = Arrays.asList(
-                ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("0", "1")),
-                ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("2", "3"))
+                ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange("0", "1")),
+                ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"),
+                        ShardObjectHelper.newHashKeyRange("2", "3"))
         );
 
         when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true);
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java
index cc03a203..4e2bae48 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java
@@ -67,7 +67,8 @@ public class ShardObjectHelper {
             String parentShardId,
             String adjacentParentShardId,
             SequenceNumberRange sequenceNumberRange) {
-        return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, HashKeyRange.builder().startingHashKey("1").endingHashKey("100").build());
+        return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange,
+                HashKeyRange.builder().startingHashKey("1").endingHashKey("100").build());
     }
 
     /** Helper method to create a new shard object.
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java
index 772aa542..d6c2d0b3 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java
@@ -141,7 +141,8 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest {
                 .withLease("5", "foo")
                 .build();
 
-        // In the current DynamoDBLeaseTaker implementation getAllLeases() gets leases from an internal cache that is built during takeLeases() operation
+        // In the current DynamoDBLeaseTaker implementation getAllLeases() gets leases from an internal cache that is
+        // built during takeLeases() operation
         assertThat(taker.allLeases().size(), equalTo(0));
 
         taker.takeLeases();
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java
index f94d82fd..c085f196 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java
@@ -40,7 +40,6 @@ import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.runners.MockitoJUnitRunner;
 
-import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
 import software.amazon.awssdk.services.kinesis.model.ChildShard;
 import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer;
 import software.amazon.kinesis.common.InitialPositionInStream;
@@ -88,12 +87,8 @@ public class ConsumerStatesTest {
     @Mock
     private ShutdownNotification shutdownNotification;
     @Mock
-    private InitialPositionInStreamExtended initialPositionInStream;
-    @Mock
     private RecordsPublisher recordsPublisher;
     @Mock
-    private KinesisAsyncClient kinesisClient;
-    @Mock
     private ShardDetector shardDetector;
     @Mock
     private HierarchicalShardSyncer hierarchicalShardSyncer;
@@ -121,7 +116,8 @@ public class ConsumerStatesTest {
 
     @Before
     public void setup() {
-        argument = new ShardConsumerArgument(shardInfo, StreamIdentifier.singleStreamInstance(STREAM_NAME), leaseCoordinator, executorService, recordsPublisher,
+        argument = new ShardConsumerArgument(shardInfo, StreamIdentifier.singleStreamInstance(STREAM_NAME),
+                leaseCoordinator, executorService, recordsPublisher,
                 shardRecordProcessor, checkpointer, recordProcessorCheckpointer, parentShardPollIntervalMillis,
                 taskBackoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, listShardsBackoffTimeInMillis,
                 maxListShardsRetryAttempts, shouldCallProcessRecordsEvenForEmptyRecordList, idleTimeInMillis,
@@ -134,7 +130,7 @@ public class ConsumerStatesTest {
         when(recordProcessorCheckpointer.checkpointer()).thenReturn(checkpointer);
     }
 
-    private static final Class LEASE_REFRESHER_CLASS = (Class) (Class) LeaseRefresher.class;
+    private static final Class LEASE_REFRESHER_CLASS = LeaseRefresher.class;
 
     @Test
     public void blockOnParentStateTest() {
@@ -431,7 +427,6 @@ public class ConsumerStatesTest {
                 }
             }
             this.matchingField = matching;
-
         }
 
         @Override
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ProcessTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ProcessTaskTest.java
index 12476837..1e89d8cc 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ProcessTaskTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ProcessTaskTest.java
@@ -101,7 +101,6 @@ public class ProcessTaskTest {
     @Mock
     private GlueSchemaRegistryDeserializer glueSchemaRegistryDeserializer;
 
-
     private static final byte[] TEST_DATA = new byte[] { 1, 2, 3, 4 };
 
     private final String shardId = "shard-test";
@@ -116,7 +115,6 @@ public class ProcessTaskTest {
 
     private ProcessTask processTask;
 
-
     @Before
     public void setUpProcessTask() {
         when(checkpointer.checkpointer()).thenReturn(mock(Checkpointer.class));
@@ -130,7 +128,8 @@ public class ProcessTaskTest {
     }
 
     private ProcessTask makeProcessTask(ProcessRecordsInput processRecordsInput, GlueSchemaRegistryDeserializer deserializer) {
-        return makeProcessTask(processRecordsInput, new AggregatorUtil(), skipShardSyncAtWorkerInitializationIfLeasesExist, new SchemaRegistryDecoder(deserializer));
+        return makeProcessTask(processRecordsInput, new AggregatorUtil(), skipShardSyncAtWorkerInitializationIfLeasesExist,
+                new SchemaRegistryDecoder(deserializer));
     }
 
     private ProcessTask makeProcessTask(ProcessRecordsInput processRecordsInput, AggregatorUtil aggregatorUtil,
@@ -149,11 +148,8 @@ public class ProcessTaskTest {
         );
     }
 
-
-
     @Test
     public void testProcessTaskWithShardEndReached() {
-
         processTask = makeProcessTask(processRecordsInput);
         when(processRecordsInput.isAtShardEnd()).thenReturn(true);
 
@@ -429,7 +425,8 @@ public class ProcessTaskTest {
 
         when(processRecordsInput.records()).thenReturn(rawRecords);
         ProcessTask processTask = makeProcessTask(processRecordsInput, aggregatorUtil, false);
-        ShardRecordProcessorOutcome outcome = testWithRecords(processTask, new ExtendedSequenceNumber(sequenceNumber.subtract(BigInteger.valueOf(100)).toString(), 0L),
+        ShardRecordProcessorOutcome outcome = testWithRecords(processTask,
+                new ExtendedSequenceNumber(sequenceNumber.subtract(BigInteger.valueOf(100)).toString(), 0L),
                 new ExtendedSequenceNumber(sequenceNumber.toString(), 0L));
 
         assertThat(outcome.processRecordsCall.records(), equalTo(expectedRecords));
@@ -645,12 +642,12 @@ public class ProcessTaskTest {
     }
 
     private ShardRecordProcessorOutcome testWithRecords(List records,
-                                                        ExtendedSequenceNumber lastCheckpointValue, ExtendedSequenceNumber largestPermittedCheckpointValue) {
+            ExtendedSequenceNumber lastCheckpointValue, ExtendedSequenceNumber largestPermittedCheckpointValue) {
         return testWithRecords(records, lastCheckpointValue, largestPermittedCheckpointValue, new AggregatorUtil());
     }
 
     private ShardRecordProcessorOutcome testWithRecords(List records, ExtendedSequenceNumber lastCheckpointValue,
-                                                        ExtendedSequenceNumber largestPermittedCheckpointValue, AggregatorUtil aggregatorUtil) {
+            ExtendedSequenceNumber largestPermittedCheckpointValue, AggregatorUtil aggregatorUtil) {
         when(processRecordsInput.records()).thenReturn(records);
         return testWithRecords(
                 makeProcessTask(processRecordsInput, aggregatorUtil, skipShardSyncAtWorkerInitializationIfLeasesExist),
@@ -658,7 +655,7 @@ public class ProcessTaskTest {
     }
 
     private ShardRecordProcessorOutcome testWithRecords(ProcessTask processTask, ExtendedSequenceNumber lastCheckpointValue,
-                                                        ExtendedSequenceNumber largestPermittedCheckpointValue) {
+            ExtendedSequenceNumber largestPermittedCheckpointValue) {
         when(checkpointer.lastCheckpointValue()).thenReturn(lastCheckpointValue);
         when(checkpointer.largestPermittedCheckpointValue()).thenReturn(largestPermittedCheckpointValue);
         processTask.call();
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
index 9615794b..fc242fed 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java
@@ -103,7 +103,7 @@ public class FanOutRecordsPublisherTest {
     private SubscribeToShardEvent batchEvent;
 
     @Test
-    public void simpleTest() throws Exception {
+    public void testSimple() {
         FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN);
 
         ArgumentCaptor captor = ArgumentCaptor
@@ -218,8 +218,10 @@ public class FanOutRecordsPublisherTest {
         List matchers = records.stream().map(KinesisClientRecordMatcher::new)
                                                            .collect(Collectors.toList());
 
-        batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).continuationSequenceNumber(CONTINUATION_SEQUENCE_NUMBER).build();
-        SubscribeToShardEvent invalidEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).childShards(Collections.emptyList()).build();
+        batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records)
+                .continuationSequenceNumber(CONTINUATION_SEQUENCE_NUMBER).build();
+        SubscribeToShardEvent invalidEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L)
+                .records(records).childShards(Collections.emptyList()).build();
 
         captor.getValue().onNext(batchEvent);
         captor.getValue().onNext(invalidEvent);
@@ -238,7 +240,7 @@ public class FanOutRecordsPublisherTest {
     }
 
     @Test
-    public void testIfAllEventsReceivedWhenNoTasksRejectedByExecutor() throws Exception {
+    public void testIfAllEventsReceivedWhenNoTasksRejectedByExecutor() {
         FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN);
 
         ArgumentCaptor captor = ArgumentCaptor
@@ -414,7 +416,8 @@ public class FanOutRecordsPublisherTest {
         int totalServicePublisherEvents = 1000;
         int initialDemand = 0;
         BackpressureAdheringServicePublisher servicePublisher =
-                new BackpressureAdheringServicePublisher(servicePublisherAction, totalServicePublisherEvents, servicePublisherTaskCompletionLatch, initialDemand);
+                new BackpressureAdheringServicePublisher(servicePublisherAction, totalServicePublisherEvents,
+                        servicePublisherTaskCompletionLatch, initialDemand);
 
         doNothing().when(publisher).subscribe(captor.capture());
 
@@ -848,7 +851,8 @@ public class FanOutRecordsPublisherTest {
         int totalServicePublisherEvents = 1000;
         int initialDemand = 9;
         BackpressureAdheringServicePublisher servicePublisher =
-                new BackpressureAdheringServicePublisher(servicePublisherAction, totalServicePublisherEvents, servicePublisherTaskCompletionLatch, initialDemand);
+                new BackpressureAdheringServicePublisher(servicePublisherAction, totalServicePublisherEvents,
+                        servicePublisherTaskCompletionLatch, initialDemand);
 
         doNothing().when(publisher).subscribe(captor.capture());
 
@@ -941,7 +945,8 @@ public class FanOutRecordsPublisherTest {
         int totalServicePublisherEvents = 1000;
         int initialDemand = 11;
         BackpressureAdheringServicePublisher servicePublisher =
-                new BackpressureAdheringServicePublisher(servicePublisherAction, totalServicePublisherEvents, servicePublisherTaskCompletionLatch, initialDemand);
+                new BackpressureAdheringServicePublisher(servicePublisherAction, totalServicePublisherEvents,
+                        servicePublisherTaskCompletionLatch, initialDemand);
 
         doNothing().when(publisher).subscribe(captor.capture());
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/AsynchronousGetRecordsRetrievalStrategyTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/AsynchronousGetRecordsRetrievalStrategyTest.java
index 2c7d8fd1..fc4b4fe1 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/AsynchronousGetRecordsRetrievalStrategyTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/AsynchronousGetRecordsRetrievalStrategyTest.java
@@ -126,7 +126,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest {
     }
 
     @Test(expected = IllegalStateException.class)
-    public void testStrategyIsShutdown() throws Exception {
+    public void testStrategyIsShutdown() {
         AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher,
                 executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID);
 
@@ -141,7 +141,8 @@ public class AsynchronousGetRecordsRetrievalStrategyTest {
                 executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID);
 
         when(executorService.isShutdown()).thenReturn(false);
-        when(completionService.submit(any())).thenReturn(blockedFuture).thenThrow(new RejectedExecutionException("Rejected!")).thenReturn(successfulFuture);
+        when(completionService.submit(any())).thenReturn(blockedFuture)
+                .thenThrow(new RejectedExecutionException("Rejected!")).thenReturn(successfulFuture);
         when(completionService.poll(anyLong(), any())).thenReturn(null).thenReturn(null).thenReturn(successfulFuture);
         when(successfulFuture.get()).thenReturn(dataFetcherResult);
         when(successfulFuture.cancel(anyBoolean())).thenReturn(false);
@@ -156,7 +157,6 @@ public class AsynchronousGetRecordsRetrievalStrategyTest {
         verify(successfulFuture).cancel(eq(true));
         verify(blockedFuture).cancel(eq(true));
 
-
         assertThat(actualResult, equalTo(expectedResponses));
     }
     
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java
index d9955da4..780ac4ad 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java
@@ -162,7 +162,8 @@ public class PrefetchRecordsPublisherIntegrationTest {
     @Test
     public void testDifferentShardCaches() {
         final ExecutorService executorService2 = spy(Executors.newFixedThreadPool(1));
-        final KinesisDataFetcher kinesisDataFetcher = spy(new KinesisDataFetcher(kinesisClient, streamName, shardId, MAX_RECORDS_PER_CALL, NULL_METRICS_FACTORY));
+        final KinesisDataFetcher kinesisDataFetcher = spy(new KinesisDataFetcher(kinesisClient, streamName, shardId,
+                MAX_RECORDS_PER_CALL, NULL_METRICS_FACTORY));
         final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy2 =
                 spy(new AsynchronousGetRecordsRetrievalStrategy(kinesisDataFetcher, 5 , 5, shardId));
         final PrefetchRecordsPublisher recordsPublisher2 = new PrefetchRecordsPublisher(
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java
index af02469a..8d88151b 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java
@@ -133,7 +133,8 @@ public class PrefetchRecordsPublisherTest {
         getRecordsCache = createPrefetchRecordsPublisher(0L);
         spyQueue = spy(getRecordsCache.getPublisherSession().prefetchRecordsQueue());
         records = spy(new ArrayList<>());
-        getRecordsResponse = GetRecordsResponse.builder().records(records).nextShardIterator(NEXT_SHARD_ITERATOR).childShards(new ArrayList<>()).build();
+        getRecordsResponse = GetRecordsResponse.builder().records(records).nextShardIterator(NEXT_SHARD_ITERATOR)
+                .childShards(Collections.emptyList()).build();
 
         when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResponse);
     }
diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml
index 76c4b330..363527bd 100644
--- a/checkstyle/checkstyle.xml
+++ b/checkstyle/checkstyle.xml
@@ -10,7 +10,7 @@
 
     
         
-        
+        
         
     
 
@@ -22,8 +22,8 @@
     
 
     
-        
         
+        
         
         
         

From c3883f576394bfda2526f5d2a8ebcb2dde488735 Mon Sep 17 00:00:00 2001
From: Meher M <134537845+mmankika-aws@users.noreply.github.com>
Date: Thu, 6 Jul 2023 16:47:15 -0700
Subject: [PATCH 81/88] Only deleting resource created by ITs (#1162)

---
 .../software/amazon/kinesis/utils/AWSResourceManager.java    | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java
index 4a6bcfaf..0d4ba656 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/AWSResourceManager.java
@@ -65,7 +65,10 @@ public abstract class AWSResourceManager {
     public void deleteAllResource() throws Exception {
         final List resourceNames = getAllResourceNames();
         for (String resourceName : resourceNames) {
-            deleteResource(resourceName);
+            // Delete all resources that have prefix "KCLRelease"
+            if (resourceName.startsWith("KCLRelease")) {
+                deleteResource(resourceName);
+            }
         }
     }
 }

From b2c3f9712ad4b0723b939181a8cf684a75f87e41 Mon Sep 17 00:00:00 2001
From: pelaezryan 
Date: Fri, 7 Jul 2023 12:35:08 -0700
Subject: [PATCH 82/88] Bumped KCL Version from 2.5.1 to 2.5.2-SNAPSHOT (#1164)

* Bumped KCL Version from 2.5.1 to 2.5.2-SNAPSHOT

* Updated pom.xml for amazon-kinesis-client and amazon-kinesis-client-multilang to 2.5.2-SNAPSHOT

---------

Co-authored-by: Ryan Pelaez 
---
 amazon-kinesis-client-multilang/pom.xml                         | 2 +-
 amazon-kinesis-client/pom.xml                                   | 2 +-
 .../java/software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +-
 pom.xml                                                         | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)

diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml
index 8494c3c8..673b3eaa 100644
--- a/amazon-kinesis-client-multilang/pom.xml
+++ b/amazon-kinesis-client-multilang/pom.xml
@@ -21,7 +21,7 @@
   
     amazon-kinesis-client-pom
     software.amazon.kinesis
-    2.5.1
+    2.5.2-SNAPSHOT
   
   4.0.0
 
diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml
index 2ba3aaac..b4f51cab 100644
--- a/amazon-kinesis-client/pom.xml
+++ b/amazon-kinesis-client/pom.xml
@@ -22,7 +22,7 @@
   
     software.amazon.kinesis
     amazon-kinesis-client-pom
-    2.5.1
+    2.5.2-SNAPSHOT
   
 
   amazon-kinesis-client
diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
index 3cb4a0d9..ba9f791f 100644
--- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
+++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java
@@ -49,7 +49,7 @@ public class RetrievalConfig {
      */
     public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java";
 
-    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.1";
+    public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.2-SNAPSHOT";
 
     /**
      * Client used to make calls to Kinesis for records retrieval
diff --git a/pom.xml b/pom.xml
index dcebcb9c..abd1308e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -22,7 +22,7 @@
   amazon-kinesis-client-pom
   pom
   Amazon Kinesis Client Library
-  2.5.1
+  2.5.2-SNAPSHOT
   The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
     from Amazon Kinesis.
   

From eccd6cf2e76c574e0bdabcca3383b0377b49ff22 Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Mon, 17 Jul 2023 16:08:27 -0400
Subject: [PATCH 83/88] CVE-2023-2976: revbump Guava `32.0.0-jre` ->
 `32.1.1-jre` (#1181)

https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2023-2976
---
 amazon-kinesis-client/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml
index b4f51cab..45681178 100644
--- a/amazon-kinesis-client/pom.xml
+++ b/amazon-kinesis-client/pom.xml
@@ -88,7 +88,7 @@
     
       com.google.guava
       guava
-      32.0.0-jre
+      32.1.1-jre
     
     
       com.google.protobuf

From 46cd1179d4ee2e5d8ce0c35985fdb6c484a0d529 Mon Sep 17 00:00:00 2001
From: Meher M <134537845+mmankika-aws@users.noreply.github.com>
Date: Thu, 3 Aug 2023 13:16:56 -0700
Subject: [PATCH 84/88] Adding resharding integration tests and changing ITs to
 not run by default (#1152)

* Initial changes for resharding integration tests KCL 2.x and changing integration tests to not run by default
---
 README.md                                     |  14 ++-
 amazon-kinesis-client/pom.xml                 |   2 +
 .../{utils => application}/TestConsumer.java  | 117 +++++++++++++++---
 .../TestRecordProcessor.java                  |   3 +-
 .../TestRecordProcessorFactory.java           |   3 +-
 .../amazon/kinesis/config/KCLAppConfig.java   |  30 +----
 ...seCanaryStreamingReshardingTestConfig.java |  34 +++++
 .../BasicStreamConsumerIntegrationTest.java   |   2 +-
 .../lifecycle/ReshardIntegrationTest.java     |  15 +++
 .../amazon/kinesis/utils/ReshardOptions.java  |  14 ++-
 .../kinesis/utils/StreamExistenceManager.java |   1 -
 11 files changed, 179 insertions(+), 56 deletions(-)
 rename amazon-kinesis-client/src/test/java/software/amazon/kinesis/{utils => application}/TestConsumer.java (64%)
 rename amazon-kinesis-client/src/test/java/software/amazon/kinesis/{utils => application}/TestRecordProcessor.java (97%)
 rename amazon-kinesis-client/src/test/java/software/amazon/kinesis/{utils => application}/TestRecordProcessorFactory.java (84%)
 create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingReshardingTestConfig.java
 create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ReshardIntegrationTest.java

diff --git a/README.md b/README.md
index 3cc92799..b6ebb6a1 100644
--- a/README.md
+++ b/README.md
@@ -32,15 +32,17 @@ Please open an issue if you have any questions.
 ## Building from Source
 
 After you've downloaded the code from GitHub, you can build it using Maven. To disable GPG signing in the build, use
- this command: `mvn clean install -Dgpg.skip=true`. Note: This command runs Integration tests, which in turn creates AWS
-  resources (which requires manual cleanup). Integration tests require valid AWS credentials need to be discovered at
-   runtime. To skip running integration tests, add ` -DskipITs` option to the build command.
+this command: `mvn clean install -Dgpg.skip=true`. 
+Note: This command does not run integration tests.
 
 ## Running Integration Tests
 
-To run integration tests: `mvn -Dit.test=*IntegrationTest verify`. 
-This will look for a default AWS profile specified in your local `.aws/credentials`. 
-Optionally, you can provide the name of an IAM user/role to run tests with as a string using this command: `mvn -Dit.test=*IntegrationTest -DawsProfile="" verify`.
+Note that running integration tests creates AWS resources.
+Integration tests require valid AWS credentials.
+This will look for a default AWS profile specified in your local `.aws/credentials`.
+To run all integration tests: `mvn verify -DskipITs=false`.
+To run one integration tests: `mvn -Dit.test=*IntegrationTest -DskipITs=false verify`
+Optionally, you can provide the name of an IAM user/role to run tests with as a string using this command: `mvn verify -DskipITs=false -DawsProfile=""`.
 
 ## Integration with the Kinesis Producer Library
 For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesis-guide-kpl]**, the KCL integrates without additional effort. When the KCL retrieves an aggregated Amazon Kinesis record consisting of multiple KPL user records, it will automatically invoke the KPL to extract the individual user records before returning them to the user.
diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml
index 45681178..653d581f 100644
--- a/amazon-kinesis-client/pom.xml
+++ b/amazon-kinesis-client/pom.xml
@@ -52,6 +52,7 @@
     ${project.build.directory}/test-lib
     2.0.7
     1.1.14
+    true
   
 
   
@@ -199,6 +200,7 @@
         maven-surefire-plugin
         2.22.2
         
+          ${skipITs}
           
             **/*IntegrationTest.java
           
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestConsumer.java
similarity index 64%
rename from amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java
rename to amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestConsumer.java
index 223ca99a..3e4e931d 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestConsumer.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestConsumer.java
@@ -1,13 +1,19 @@
-package software.amazon.kinesis.utils;
+package software.amazon.kinesis.application;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import lombok.Data;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.commons.lang3.RandomStringUtils;
 import software.amazon.awssdk.core.SdkBytes;
 import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
 import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryRequest;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse;
 import software.amazon.awssdk.services.kinesis.model.PutRecordRequest;
+import software.amazon.awssdk.services.kinesis.model.ScalingType;
+import software.amazon.awssdk.services.kinesis.model.UpdateShardCountRequest;
+import software.amazon.awssdk.services.kinesis.model.UpdateShardCountResponse;
 import software.amazon.kinesis.checkpoint.CheckpointConfig;
 import software.amazon.kinesis.common.ConfigsBuilder;
 import software.amazon.kinesis.common.InitialPositionInStreamExtended;
@@ -19,9 +25,14 @@ import software.amazon.kinesis.lifecycle.LifecycleConfig;
 import software.amazon.kinesis.metrics.MetricsConfig;
 import software.amazon.kinesis.processor.ProcessorConfig;
 import software.amazon.kinesis.retrieval.RetrievalConfig;
+import software.amazon.kinesis.utils.LeaseTableManager;
+import software.amazon.kinesis.utils.RecordValidationStatus;
+import software.amazon.kinesis.utils.ReshardOptions;
+import software.amazon.kinesis.utils.StreamExistenceManager;
 
 import java.math.BigInteger;
 import java.nio.ByteBuffer;
+import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -78,8 +89,10 @@ public class TestConsumer {
         try {
             startConsumer();
 
-            // Sleep for three minutes to allow the producer/consumer to run and then end the test case.
-            Thread.sleep(TimeUnit.SECONDS.toMillis(60 * 3));
+            // Sleep to allow the producer/consumer to run and then end the test case.
+            // If non-reshard sleep 3 minutes, else sleep 4 minutes per scale.
+            final int sleepMinutes = (consumerConfig.getReshardFactorList() == null) ? 3 : (4 * consumerConfig.getReshardFactorList().size());
+            Thread.sleep(TimeUnit.MINUTES.toMillis(sleepMinutes));
 
             // Stops sending dummy data.
             stopProducer();
@@ -115,9 +128,25 @@ public class TestConsumer {
     }
 
     private void startProducer() {
-        // Send dummy data to stream
         this.producerExecutor = Executors.newSingleThreadScheduledExecutor();
-        this.producerFuture = producerExecutor.scheduleAtFixedRate(this::publishRecord, 60, 1, TimeUnit.SECONDS);
+        this.producerFuture = producerExecutor.scheduleAtFixedRate(this::publishRecord, 10, 1, TimeUnit.SECONDS);
+
+        // Reshard logic if required for the test
+        if (consumerConfig.getReshardFactorList() != null) {
+            log.info("----Reshard Config found: {}", consumerConfig.getReshardFactorList());
+
+            final StreamScaler s = new StreamScaler(
+                    kinesisClient,
+                    consumerConfig.getStreamName(),
+                    consumerConfig.getReshardFactorList(),
+                    consumerConfig
+            );
+
+            // Schedule the stream scales 4 minutes apart with 2 minute starting delay
+            for (int i = 0; i < consumerConfig.getReshardFactorList().size(); i++) {
+                producerExecutor.schedule(s, (4 * i) + 2, TimeUnit.MINUTES);
+            }
+        }
     }
 
     private void setUpConsumerResources() throws Exception {
@@ -128,7 +157,9 @@ public class TestConsumer {
         checkpointConfig = configsBuilder.checkpointConfig();
         coordinatorConfig = configsBuilder.coordinatorConfig();
         leaseManagementConfig = configsBuilder.leaseManagementConfig()
-                .initialPositionInStream(InitialPositionInStreamExtended.newInitialPosition(consumerConfig.getInitialPosition()))
+                .initialPositionInStream(
+                    InitialPositionInStreamExtended.newInitialPosition(consumerConfig.getInitialPosition())
+                )
                 .initialLeaseTableReadCapacity(50).initialLeaseTableWriteCapacity(50);
         lifecycleConfig = configsBuilder.lifecycleConfig();
         processorConfig = configsBuilder.processorConfig();
@@ -152,6 +183,16 @@ public class TestConsumer {
         this.consumerFuture = consumerExecutor.schedule(scheduler, 0, TimeUnit.SECONDS);
     }
 
+    private void stopProducer() {
+        log.info("Cancelling producer and shutting down executor.");
+        if (producerFuture != null) {
+            producerFuture.cancel(false);
+        }
+        if (producerExecutor != null) {
+            producerExecutor.shutdown();
+        }
+    }
+
     public void publishRecord() {
         final PutRecordRequest request;
         try {
@@ -175,7 +216,7 @@ public class TestConsumer {
 
     private ByteBuffer wrapWithCounter(int payloadSize, BigInteger payloadCounter) throws RuntimeException {
         final byte[] returnData;
-        log.info("--------------Putting record with data: {}", payloadCounter);
+        log.info("---------Putting record with data: {}", payloadCounter);
         try {
             returnData = mapper.writeValueAsBytes(payloadCounter);
         } catch (Exception e) {
@@ -184,12 +225,6 @@ public class TestConsumer {
         return ByteBuffer.wrap(returnData);
     }
 
-    private void stopProducer() {
-        log.info("Cancelling producer and shutting down executor.");
-        producerFuture.cancel(false);
-        producerExecutor.shutdown();
-    }
-
     private void awaitConsumerFinish() throws Exception {
         Future gracefulShutdownFuture = scheduler.startGracefulShutdown();
         log.info("Waiting up to 20 seconds for shutdown to complete.");
@@ -198,7 +233,7 @@ public class TestConsumer {
         } catch (InterruptedException e) {
             log.info("Interrupted while waiting for graceful shutdown. Continuing.");
         } catch (ExecutionException | TimeoutException e) {
-            throw e;
+            scheduler.shutdown();
         }
         log.info("Completed, shutting down now.");
     }
@@ -209,15 +244,61 @@ public class TestConsumer {
         if (errorVal != RecordValidationStatus.NO_ERROR) {
             throw new RuntimeException("There was an error validating the records that were processed: " + errorVal.toString());
         }
-        log.info("--------------Completed validation of processed records.--------------");
+        log.info("---------Completed validation of processed records.---------");
     }
 
     private void deleteResources(StreamExistenceManager streamExistenceManager, LeaseTableManager leaseTableManager) throws Exception {
-        log.info("-------------Start deleting stream.----------------");
+        log.info("-------------Start deleting stream.---------");
         streamExistenceManager.deleteResource(this.streamName);
-        log.info("-------------Start deleting lease table.----------------");
+        log.info("---------Start deleting lease table.---------");
         leaseTableManager.deleteResource(this.consumerConfig.getStreamName());
-        log.info("-------------Finished deleting resources.----------------");
+        log.info("---------Finished deleting resources.---------");
     }
 
+    @Data
+    private static class StreamScaler implements Runnable {
+        private final KinesisAsyncClient client;
+        private final String streamName;
+        private final List scalingFactors;
+        private final KCLAppConfig consumerConfig;
+        private int scalingFactorIdx = 0;
+        private DescribeStreamSummaryRequest describeStreamSummaryRequest;
+
+        private synchronized void scaleStream() throws InterruptedException, ExecutionException {
+            final DescribeStreamSummaryResponse response = client.describeStreamSummary(describeStreamSummaryRequest).get();
+
+            final int openShardCount = response.streamDescriptionSummary().openShardCount();
+            final int targetShardCount = scalingFactors.get(scalingFactorIdx).calculateShardCount(openShardCount);
+
+            log.info("Scaling stream {} from {} shards to {} shards w/ scaling factor {}",
+                    streamName, openShardCount, targetShardCount, scalingFactors.get(scalingFactorIdx));
+
+            final UpdateShardCountRequest updateShardCountRequest = UpdateShardCountRequest.builder()
+                    .streamName(streamName).targetShardCount(targetShardCount).scalingType(ScalingType.UNIFORM_SCALING).build();
+            final UpdateShardCountResponse shardCountResponse = client.updateShardCount(updateShardCountRequest).get();
+            log.info("Executed shard scaling request. Response Details : {}", shardCountResponse.toString());
+
+            scalingFactorIdx++;
+        }
+
+        @Override
+        public void run() {
+            if (scalingFactors.size() == 0 || scalingFactorIdx >= scalingFactors.size()) {
+                log.info("No scaling factor found in list");
+                return;
+            }
+            log.info("Starting stream scaling with params : {}", this);
+
+            if (describeStreamSummaryRequest == null) {
+                describeStreamSummaryRequest = DescribeStreamSummaryRequest.builder().streamName(streamName).build();
+            }
+            try {
+                scaleStream();
+            } catch (InterruptedException | ExecutionException e) {
+                log.error("Caught error while scaling shards for stream", e);
+            } finally {
+                log.info("Reshard List State : {}", scalingFactors);
+            }
+        }
+    }
 }
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestRecordProcessor.java
similarity index 97%
rename from amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java
rename to amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestRecordProcessor.java
index f3e43915..0e4dc489 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessor.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestRecordProcessor.java
@@ -1,4 +1,4 @@
-package software.amazon.kinesis.utils;
+package software.amazon.kinesis.application;
 
 import lombok.extern.slf4j.Slf4j;
 import org.slf4j.MDC;
@@ -11,6 +11,7 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput;
 import software.amazon.kinesis.processor.ShardRecordProcessor;
 import software.amazon.kinesis.lifecycle.events.InitializationInput;
 import software.amazon.kinesis.retrieval.KinesisClientRecord;
+import software.amazon.kinesis.utils.RecordValidatorQueue;
 
 import java.nio.ByteBuffer;
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestRecordProcessorFactory.java
similarity index 84%
rename from amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java
rename to amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestRecordProcessorFactory.java
index 03361b6e..4e06890e 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/TestRecordProcessorFactory.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/application/TestRecordProcessorFactory.java
@@ -1,7 +1,8 @@
-package software.amazon.kinesis.utils;
+package software.amazon.kinesis.application;
 
 import software.amazon.kinesis.processor.ShardRecordProcessor;
 import software.amazon.kinesis.processor.ShardRecordProcessorFactory;
+import software.amazon.kinesis.utils.RecordValidatorQueue;
 
 public class TestRecordProcessorFactory implements ShardRecordProcessorFactory {
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java
index 5365ca4f..b5d0c4d1 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/KCLAppConfig.java
@@ -5,7 +5,7 @@ import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
 import software.amazon.kinesis.common.InitialPositionInStreamExtended;
 import software.amazon.kinesis.utils.RecordValidatorQueue;
 import software.amazon.kinesis.utils.ReshardOptions;
-import software.amazon.kinesis.utils.TestRecordProcessorFactory;
+import software.amazon.kinesis.application.TestRecordProcessorFactory;
 import lombok.Builder;
 import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider;
 import software.amazon.awssdk.auth.credentials.ProfileCredentialsProvider;
@@ -29,6 +29,7 @@ import java.io.IOException;
 import java.net.Inet4Address;
 import java.net.URISyntaxException;
 import java.net.UnknownHostException;
+import java.util.List;
 
 /**
  * Default configuration for a producer or consumer used in integration tests.
@@ -75,7 +76,7 @@ public abstract class KCLAppConfig {
                 .build();
     }
 
-    public ReshardConfig getReshardConfig() {
+    public List getReshardFactorList() {
         return null;
     }
 
@@ -157,34 +158,11 @@ public abstract class KCLAppConfig {
      */
     @Value
     @Builder
-    static class ProducerConfig {
+    public static class ProducerConfig {
         private boolean isBatchPut;
         private int batchSize;
         private int recordSizeKB;
         private long callPeriodMills;
     }
 
-    /**
-     * Description of the method of resharding for a test case
-     */
-    @Value
-    @Builder
-    static class ReshardConfig {
-        /**
-         * reshardingFactorCycle: lists the order or reshards that will be done during one reshard cycle
-         * e.g {SPLIT, MERGE} means that the number of shards will first be doubled, then halved
-         */
-        private ReshardOptions[] reshardingFactorCycle;
-
-        /**
-         * numReshardCycles: the number of resharding cycles that will be executed in a test
-         */
-        private int numReshardCycles;
-
-        /**
-         * reshardFrequencyMillis: the period of time between reshard cycles (in milliseconds)
-         */
-        private long reshardFrequencyMillis;
-    }
-
 }
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingReshardingTestConfig.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingReshardingTestConfig.java
new file mode 100644
index 00000000..cfdc5298
--- /dev/null
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/config/ReleaseCanaryStreamingReshardingTestConfig.java
@@ -0,0 +1,34 @@
+package software.amazon.kinesis.config;
+
+import software.amazon.awssdk.http.Protocol;
+import software.amazon.kinesis.utils.ReshardOptions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+import static software.amazon.kinesis.utils.ReshardOptions.MERGE;
+import static software.amazon.kinesis.utils.ReshardOptions.SPLIT;
+
+public class ReleaseCanaryStreamingReshardingTestConfig extends KCLAppConfig {
+
+    private final UUID uniqueId = UUID.randomUUID();
+    @Override
+    public String getStreamName() {
+        return "KCLReleaseCanary2XStreamingReshardingTestStream_" + uniqueId;
+    }
+
+    @Override
+    public Protocol getKinesisClientProtocol() { return Protocol.HTTP2; }
+
+    @Override
+    public int getShardCount() {
+        return 100;
+    }
+
+    @Override
+    public List getReshardFactorList() {
+        return Arrays.asList(SPLIT, MERGE);
+    }
+
+}
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java
index e2e44687..d03254c2 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BasicStreamConsumerIntegrationTest.java
@@ -5,7 +5,7 @@ import software.amazon.kinesis.config.KCLAppConfig;
 import software.amazon.kinesis.config.ReleaseCanaryPollingH2TestConfig;
 import software.amazon.kinesis.config.ReleaseCanaryPollingH1TestConfig;
 import software.amazon.kinesis.config.ReleaseCanaryStreamingTestConfig;
-import software.amazon.kinesis.utils.TestConsumer;
+import software.amazon.kinesis.application.TestConsumer;
 
 public class BasicStreamConsumerIntegrationTest {
 
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ReshardIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ReshardIntegrationTest.java
new file mode 100644
index 00000000..aa08980e
--- /dev/null
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ReshardIntegrationTest.java
@@ -0,0 +1,15 @@
+package software.amazon.kinesis.lifecycle;
+
+import org.junit.Test;
+import software.amazon.kinesis.config.KCLAppConfig;
+import software.amazon.kinesis.config.ReleaseCanaryStreamingReshardingTestConfig;
+import software.amazon.kinesis.application.TestConsumer;
+
+public class ReshardIntegrationTest {
+    @Test
+    public void kclReleaseCanaryStreamingReshardingTest() throws Exception {
+        KCLAppConfig consumerConfig = new ReleaseCanaryStreamingReshardingTestConfig();
+        TestConsumer consumer = new TestConsumer(consumerConfig);
+        consumer.run();
+    }
+}
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java
index fbf5f68b..f1513cfb 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ReshardOptions.java
@@ -6,6 +6,16 @@ package software.amazon.kinesis.utils;
  * Merge halves the number of shards.
  */
 public enum ReshardOptions {
-    SPLIT,
-    MERGE
+    SPLIT {
+        public int calculateShardCount(int currentShards) {
+            return (int) (2.0 * currentShards);
+        }
+    },
+    MERGE {
+        public int calculateShardCount(int currentShards) {
+            return (int) (0.5 * currentShards);
+        }
+    };
+
+    public abstract int calculateShardCount(int currentShards);
 }
diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java
index b5f06b78..db8615c3 100644
--- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java
+++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/StreamExistenceManager.java
@@ -111,5 +111,4 @@ public class StreamExistenceManager extends AWSResourceManager {
             }
         }
     }
-
 }

From 2f4ff65681468dc6a51f55852a983d049cc4aeec Mon Sep 17 00:00:00 2001
From: stair <123031771+stair-aws@users.noreply.github.com>
Date: Mon, 7 Aug 2023 16:29:49 -0400
Subject: [PATCH 85/88] =?UTF-8?q?[#367]=20Enhanced=20multi-lang=20`AWSCred?=
 =?UTF-8?q?entialsProvider=3D...`=20decoder=20and=20c=E2=80=A6=20(#1184)?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit

* [#367] Enhanced multi-lang `AWSCredentialsProvider=...` decoder and construction.

+ added support for external ids (issue #367)
+ added support for endpoint+region (e.g., STS via VPC)

* Multiple multi-lang edits to introduce logging and additional tests.

+ added `ENDPOINT_REGION` nested key for a simpler Cx experience
+ deduplicated, and improved, logic w.r.t. CredentialsProvider
construction to NOT swallow Exceptions

* Relocated `multilang.properties` from `main/resources` to `test/resources`
---
 .../multilang/MultiLangDaemonConfig.java      |  11 +-
 .../MultiLangShardRecordProcessor.java        |  18 +--
 .../kinesis/multilang/NestedPropertyKey.java  | 145 ++++++++++++++++++
 .../multilang/NestedPropertyProcessor.java    |  54 +++++++
 ...SAssumeRoleSessionCredentialsProvider.java |  89 +++++++++++
 ...edentialsProviderPropertyValueDecoder.java | 124 ++++++++++-----
 .../multilang/config/BuilderDynaBean.java     |   4 +-
 .../config/KinesisClientLibConfigurator.java  |   1 +
 .../multilang/MultiLangDaemonConfigTest.java  |  17 +-
 .../multilang/NestedPropertyKeyTest.java      | 112 ++++++++++++++
 ...umeRoleSessionCredentialsProviderTest.java |  71 +++++++++
 ...tialsProviderPropertyValueDecoderTest.java |  61 +++++++-
 .../src/test/resources/multilang.properties   |  93 +++++++++++
 13 files changed, 734 insertions(+), 66 deletions(-)
 create mode 100644 amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyKey.java
 create mode 100644 amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyProcessor.java
 create mode 100644 amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProvider.java
 create mode 100644 amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/NestedPropertyKeyTest.java
 create mode 100644 amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProviderTest.java
 create mode 100644 amazon-kinesis-client-multilang/src/test/resources/multilang.properties

diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangDaemonConfig.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangDaemonConfig.java
index 4d3a408f..c7f77c19 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangDaemonConfig.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangDaemonConfig.java
@@ -45,11 +45,11 @@ public class MultiLangDaemonConfig {
     private static final String PROP_PROCESSING_LANGUAGE = "processingLanguage";
     private static final String PROP_MAX_ACTIVE_THREADS = "maxActiveThreads";
 
-    private MultiLangDaemonConfiguration multiLangDaemonConfiguration;
+    private final MultiLangDaemonConfiguration multiLangDaemonConfiguration;
 
-    private ExecutorService executorService;
+    private final ExecutorService executorService;
 
-    private MultiLangRecordProcessorFactory recordProcessorFactory;
+    private final MultiLangRecordProcessorFactory recordProcessorFactory;
 
     /**
      * Constructor.
@@ -165,7 +165,6 @@ public class MultiLangDaemonConfig {
                 propertyStream.close();
             }
         }
-
     }
 
     private static boolean validateProperties(Properties properties) {
@@ -182,12 +181,12 @@ public class MultiLangDaemonConfig {
         log.debug("Value for {} property is {}", PROP_MAX_ACTIVE_THREADS, maxActiveThreads);
         if (maxActiveThreads <= 0) {
             log.info("Using a cached thread pool.");
-            return new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue(),
+            return new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<>(),
                     builder.build());
         } else {
             log.info("Using a fixed thread pool with {} max active threads.", maxActiveThreads);
             return new ThreadPoolExecutor(maxActiveThreads, maxActiveThreads, 0L, TimeUnit.MILLISECONDS,
-                    new LinkedBlockingQueue(), builder.build());
+                    new LinkedBlockingQueue<>(), builder.build());
         }
     }
 
diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangShardRecordProcessor.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangShardRecordProcessor.java
index 7b0eefe2..241ea8ee 100644
--- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangShardRecordProcessor.java
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/MultiLangShardRecordProcessor.java
@@ -32,7 +32,6 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput;
 import software.amazon.kinesis.multilang.config.MultiLangDaemonConfiguration;
 import software.amazon.kinesis.processor.ShardRecordProcessor;
 
-
 /**
  * A record processor that manages creating a child process that implements the multi language protocol and connecting
  * that child process's input and outputs to a {@link MultiLangProtocol} object and calling the appropriate methods on
@@ -50,20 +49,20 @@ public class MultiLangShardRecordProcessor implements ShardRecordProcessor {
 
     private Future stderrReadTask;
 
-    private MessageWriter messageWriter;
-    private MessageReader messageReader;
-    private DrainChildSTDERRTask readSTDERRTask;
+    private final MessageWriter messageWriter;
+    private final MessageReader messageReader;
+    private final DrainChildSTDERRTask readSTDERRTask;
 
-    private ProcessBuilder processBuilder;
+    private final ProcessBuilder processBuilder;
     private Process process;
-    private ExecutorService executorService;
+    private final ExecutorService executorService;
     private ProcessState state;
 
-    private ObjectMapper objectMapper;
+    private final ObjectMapper objectMapper;
 
     private MultiLangProtocol protocol;
 
-    private MultiLangDaemonConfiguration configuration;
+    private final MultiLangDaemonConfiguration configuration;
 
     @Override
     public void initialize(InitializationInput initializationInput) {
@@ -213,7 +212,6 @@ public class MultiLangShardRecordProcessor implements ShardRecordProcessor {
         this.readSTDERRTask = readSTDERRTask;
         this.configuration = configuration;
 
-
         this.state = ProcessState.ACTIVE;
     }
 
@@ -303,8 +301,6 @@ public class MultiLangShardRecordProcessor implements ShardRecordProcessor {
 
     /**
      * We provide a package level method for unit testing this call to exit.
-     * 
-     * @param val exit value
      */
     void exit() {
         System.exit(EXIT_VALUE);
diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyKey.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyKey.java
new file mode 100644
index 00000000..ea3db8c3
--- /dev/null
+++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyKey.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2023 Amazon.com, Inc. or its affiliates.
+ * Licensed under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package software.amazon.kinesis.multilang;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.amazonaws.regions.Regions;
+import com.google.common.base.CaseFormat;
+
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * Key-Value pairs which may be nested in, and extracted from, a property value
+ * in a Java properties file. For example, given the line in a property file of
+ * {@code my_key = my_value|foo=bar} and a delimiter split on {@code |} (pipe),
+ * the value {@code my_value|foo=bar} would have a nested key of {@code foo}
+ * and its corresponding value is {@code bar}.
+ * 

+ * The order of nested properties does not matter, and these properties are optional. + * Customers may choose to provide, in any order, zero-or-more nested properties. + *

+ * Duplicate keys are not supported, and may result in a last-write-wins outcome. + */ +@Slf4j +public enum NestedPropertyKey { + + /** + * Specify the service endpoint where requests will be submitted. + * This property's value must be in the following format: + *
+     *     ENDPOINT ::= SERVICE_ENDPOINT "^" SIGNING_REGION
+     *     SERVICE_ENDPOINT ::= URL
+     *     SIGNING_REGION ::= AWS_REGION
+     * 
+ * + * It would be redundant to provide both this and {@link #ENDPOINT_REGION}. + * + * @see #ENDPOINT_REGION + * @see AWS Service endpoints + * @see Available Regions + */ + ENDPOINT { + void visit(final NestedPropertyProcessor processor, final String endpoint) { + final String[] tokens = endpoint.split("\\^"); + if (tokens.length != 2) { + throw new IllegalArgumentException("Invalid " + name() + ": " + endpoint); + } + processor.acceptEndpoint(tokens[0], tokens[1]); + } + }, + + /** + * Specify the region where service requests will be submitted. This + * region will determine both the service endpoint and signing region. + *

+ * It would be redundant to provide both this and {@link #ENDPOINT}. + * + * @see #ENDPOINT + * @see Available Regions + */ + ENDPOINT_REGION { + void visit(final NestedPropertyProcessor processor, final String region) { + processor.acceptEndpointRegion(Regions.fromName(region)); + } + }, + + /** + * External ids may be used when delegating access in a multi-tenant + * environment, or to third parties. + * + * @see + * How to use an external ID when granting access to your AWS resources to a third party + */ + EXTERNAL_ID { + void visit(final NestedPropertyProcessor processor, final String externalId) { + processor.acceptExternalId(externalId); + } + }, + + ; + + /** + * Nested key within the property value. For example, a nested key-value + * of {@code foo=bar} has a nested key of {@code foo}. + */ + @Getter(AccessLevel.PACKAGE) + private final String nestedKey; + + NestedPropertyKey() { + // convert the enum from UPPER_SNAKE_CASE to lowerCamelCase + nestedKey = CaseFormat.UPPER_UNDERSCORE.to(CaseFormat.LOWER_CAMEL, name()); + } + + abstract void visit(NestedPropertyProcessor processor, String value); + + /** + * Parses any number of parameters. Each nested property will prompt a + * visit to the {@code processor}. + * + * @param processor processor to be invoked for every nested property + * @param params parameters to check for a nested property key + */ + public static void parse(final NestedPropertyProcessor processor, final String... params) { + // Construct a disposable cache to keep this O(n). Since parsing is + // usually one-and-done, it's wasteful to maintain this cache in perpetuity. + final Map cachedKeys = new HashMap<>(); + for (final NestedPropertyKey npk : values()) { + cachedKeys.put(npk.getNestedKey(), npk); + } + + for (final String param : params) { + if (param != null) { + final String[] tokens = param.split("="); + if (tokens.length == 2) { + final NestedPropertyKey npk = cachedKeys.get(tokens[0]); + if (npk != null) { + npk.visit(processor, tokens[1]); + } else { + log.warn("Unsupported nested key: {}", param); + } + } else if (tokens.length > 2) { + log.warn("Malformed nested key: {}", param); + } else { + log.info("Parameter is not a nested key: {}", param); + } + } + } + } + +} diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyProcessor.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyProcessor.java new file mode 100644 index 00000000..d3dd7a6f --- /dev/null +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyProcessor.java @@ -0,0 +1,54 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.multilang; + +import com.amazonaws.regions.Regions; + +/** + * Defines methods to process {@link NestedPropertyKey}s. + */ +public interface NestedPropertyProcessor { + + /** + * Set the service endpoint where requests are sent. + * + * @param serviceEndpoint the service endpoint either with or without the protocol + * (e.g., https://sns.us-west-1.amazonaws.com, sns.us-west-1.amazonaws.com) + * @param signingRegion the region to use for SigV4 signing of requests (e.g. us-west-1) + * + * @see #acceptEndpointRegion(Regions) + * @see + * AwsClientBuilder.EndpointConfiguration + */ + void acceptEndpoint(String serviceEndpoint, String signingRegion); + + /** + * Set the service endpoint where requests are sent. + * + * @param region Region to be used by the client. This will be used to determine both the service endpoint + * (e.g., https://sns.us-west-1.amazonaws.com) and signing region (e.g., us-west-1) for requests. + * + * @see #acceptEndpoint(String, String) + */ + void acceptEndpointRegion(Regions region); + + /** + * Set the external id, an optional field to designate who can assume an IAM role. + * + * @param externalId external id used in the service call used to retrieve session credentials + */ + void acceptExternalId(String externalId); + +} diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProvider.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProvider.java new file mode 100644 index 00000000..3b196b94 --- /dev/null +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProvider.java @@ -0,0 +1,89 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.multilang.auth; + +import java.util.Arrays; + +import com.amazonaws.auth.AWSSessionCredentials; +import com.amazonaws.auth.AWSSessionCredentialsProvider; +import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider; +import com.amazonaws.auth.STSAssumeRoleSessionCredentialsProvider.Builder; +import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.securitytoken.AWSSecurityTokenService; +import com.amazonaws.services.securitytoken.AWSSecurityTokenServiceClient; + +import software.amazon.kinesis.multilang.NestedPropertyKey; +import software.amazon.kinesis.multilang.NestedPropertyProcessor; + +/** + * An {@link AWSSessionCredentialsProvider} that is backed by STSAssumeRole. + */ +public class KclSTSAssumeRoleSessionCredentialsProvider + implements AWSSessionCredentialsProvider, NestedPropertyProcessor { + + private final Builder builder; + + private final STSAssumeRoleSessionCredentialsProvider provider; + + /** + * + * @param params vararg parameters which must include roleArn at index=0, + * and roleSessionName at index=1 + */ + public KclSTSAssumeRoleSessionCredentialsProvider(final String[] params) { + this(params[0], params[1], Arrays.copyOfRange(params, 2, params.length)); + } + + public KclSTSAssumeRoleSessionCredentialsProvider(final String roleArn, final String roleSessionName, + final String... params) { + builder = new Builder(roleArn, roleSessionName); + NestedPropertyKey.parse(this, params); + provider = builder.build(); + } + + @Override + public AWSSessionCredentials getCredentials() { + return provider.getCredentials(); + } + + @Override + public void refresh() { + // do nothing + } + + @Override + public void acceptEndpoint(final String serviceEndpoint, final String signingRegion) { + final EndpointConfiguration endpoint = new EndpointConfiguration(serviceEndpoint, signingRegion); + final AWSSecurityTokenService stsClient = AWSSecurityTokenServiceClient.builder() + .withEndpointConfiguration(endpoint) + .build(); + builder.withStsClient(stsClient); + } + + @Override + public void acceptEndpointRegion(final Regions region) { + final AWSSecurityTokenService stsClient = AWSSecurityTokenServiceClient.builder() + .withRegion(region) + .build(); + builder.withStsClient(stsClient); + } + + @Override + public void acceptExternalId(final String externalId) { + builder.withExternalId(externalId); + } + +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java index 97fa975e..f11ac0ec 100644 --- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoder.java @@ -14,10 +14,13 @@ */ package software.amazon.kinesis.multilang.config; -import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSCredentialsProviderChain; @@ -28,7 +31,6 @@ import lombok.extern.slf4j.Slf4j; */ @Slf4j class AWSCredentialsProviderPropertyValueDecoder implements IPropertyValueDecoder { - private static final String AUTH_PREFIX = "com.amazonaws.auth."; private static final String LIST_DELIMITER = ","; private static final String ARG_DELIMITER = "|"; @@ -63,35 +65,59 @@ class AWSCredentialsProviderPropertyValueDecoder implements IPropertyValueDecode */ @Override public List> getSupportedTypes() { - return Arrays.asList(AWSCredentialsProvider.class); + return Collections.singletonList(AWSCredentialsProvider.class); } - /* + /** * Convert string list to a list of valid credentials providers. */ private static List getValidCredentialsProviders(List providerNames) { List credentialsProviders = new ArrayList<>(); + for (String providerName : providerNames) { - if (providerName.contains(ARG_DELIMITER)) { - String[] nameAndArgs = providerName.split("\\" + ARG_DELIMITER); - Class[] argTypes = new Class[nameAndArgs.length - 1]; - Arrays.fill(argTypes, String.class); - try { - Class className = Class.forName(nameAndArgs[0]); - Constructor c = className.getConstructor(argTypes); - credentialsProviders.add((AWSCredentialsProvider) c - .newInstance(Arrays.copyOfRange(nameAndArgs, 1, nameAndArgs.length))); - } catch (Exception e) { - log.debug("Can't find any credentials provider matching {}.", providerName); + final String[] nameAndArgs = providerName.split("\\" + ARG_DELIMITER); + final Class clazz; + try { + final Class c = Class.forName(nameAndArgs[0]); + if (!AWSCredentialsProvider.class.isAssignableFrom(c)) { + continue; } - } else { - try { - Class className = Class.forName(providerName); - credentialsProviders.add((AWSCredentialsProvider) className.newInstance()); - } catch (Exception e) { - log.debug("Can't find any credentials provider matching {}.", providerName); + clazz = (Class) c; + } catch (ClassNotFoundException cnfe) { + // Providers are a product of prefixed Strings to cover multiple + // namespaces (e.g., "Foo" -> { "some.auth.Foo", "kcl.auth.Foo" }). + // It's expected that many class names will not resolve. + continue; + } + log.info("Attempting to construct {}", clazz); + + AWSCredentialsProvider provider = null; + if (nameAndArgs.length > 1) { + final String[] varargs = Arrays.copyOfRange(nameAndArgs, 1, nameAndArgs.length); + + // attempt to invoke an explicit N-arg constructor of FooClass(String, String, ...) + provider = constructProvider(providerName, () -> { + Class[] argTypes = new Class[nameAndArgs.length - 1]; + Arrays.fill(argTypes, String.class); + return clazz.getConstructor(argTypes).newInstance(varargs); + }); + + if (provider == null) { + // attempt to invoke a public varargs/array constructor of FooClass(String[]) + provider = constructProvider(providerName, () -> + clazz.getConstructor(String[].class).newInstance((Object) varargs) + ); } } + + if (provider == null) { + // regardless of parameters, fallback to invoke a public no-arg constructor + provider = constructProvider(providerName, clazz::newInstance); + } + + if (provider != null) { + credentialsProviders.add(provider); + } } return credentialsProviders; } @@ -99,7 +125,7 @@ class AWSCredentialsProviderPropertyValueDecoder implements IPropertyValueDecode private static List getProviderNames(String property) { // assume list delimiter is "," String[] elements = property.split(LIST_DELIMITER); - List result = new ArrayList(); + List result = new ArrayList<>(); for (int i = 0; i < elements.length; i++) { String string = elements[i].trim(); if (!string.isEmpty()) { @@ -110,20 +136,48 @@ class AWSCredentialsProviderPropertyValueDecoder implements IPropertyValueDecode return result; } - private static List getPossibleFullClassNames(String s) { - /* - * We take care of three cases : - * - * 1. Customer provides a short name of common providers in com.amazonaws.auth package i.e. any classes - * implementing the AWSCredentialsProvider interface: - * http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html - * - * 2. Customer provides a full name of common providers e.g. com.amazonaws.auth.ClasspathFileCredentialsProvider - * - * 3. Customer provides a custom credentials provider with full name of provider - */ + private static List getPossibleFullClassNames(final String provider) { + return Stream.of( + // Customer provides a short name of common providers in com.amazonaws.auth package + // (e.g., any classes implementing the AWSCredentialsProvider interface) + // @see http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/AWSCredentialsProvider.html + "com.amazonaws.auth.", - return Arrays.asList(s, AUTH_PREFIX + s); + // Customer provides a short name of a provider offered by this multi-lang package + "software.amazon.kinesis.multilang.auth.", + + // Customer provides a fully-qualified provider name, or a custom credentials provider + // (e.g., com.amazonaws.auth.ClasspathFileCredentialsProvider, org.mycompany.FooProvider) + "" + ).map(prefix -> prefix + provider).collect(Collectors.toList()); + } + + @FunctionalInterface + private interface CredentialsProviderConstructor { + T construct() throws IllegalAccessException, InstantiationException, + InvocationTargetException, NoSuchMethodException; + } + + /** + * Attempts to construct an {@link AWSCredentialsProvider}. + * + * @param providerName Raw, unmodified provider name. Should there be an + * Exeception during construction, this parameter will be logged. + * @param constructor supplier-like function that will perform the construction + * @return the constructed provider, if successful; otherwise, null + * + * @param type of the CredentialsProvider to construct + */ + private static T constructProvider( + final String providerName, final CredentialsProviderConstructor constructor) { + try { + return constructor.construct(); + } catch (NoSuchMethodException ignored) { + // ignore + } catch (IllegalAccessException | InstantiationException | InvocationTargetException | RuntimeException e) { + log.warn("Failed to construct {}", providerName, e); + } + return null; } } diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java index 2035695c..5baa47f4 100644 --- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/BuilderDynaBean.java @@ -57,10 +57,10 @@ public class BuilderDynaBean implements DynaBean { } public BuilderDynaBean(Class destinedClass, ConvertUtilsBean convertUtilsBean, - Function emtpyPropertyHandler, List classPrefixSearchList) { + Function emptyPropertyHandler, List classPrefixSearchList) { this.convertUtilsBean = convertUtilsBean; this.classPrefixSearchList = classPrefixSearchList; - this.emptyPropertyHandler = emtpyPropertyHandler; + this.emptyPropertyHandler = emptyPropertyHandler; initialize(destinedClass); } diff --git a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java index 5e2ddb1d..49856aa6 100644 --- a/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java +++ b/amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/config/KinesisClientLibConfigurator.java @@ -62,6 +62,7 @@ public class KinesisClientLibConfigurator { public MultiLangDaemonConfiguration getConfiguration(Properties properties) { properties.entrySet().forEach(e -> { try { + log.info("Processing (key={}, value={})", e.getKey(), e.getValue()); utilsBean.setProperty(configuration, (String) e.getKey(), e.getValue()); } catch (IllegalAccessException | InvocationTargetException ex) { throw new RuntimeException(ex); diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java index c5740a2f..aa46f431 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/MultiLangDaemonConfigTest.java @@ -36,7 +36,7 @@ import software.amazon.kinesis.multilang.config.MultiLangDaemonConfiguration; @RunWith(MockitoJUnitRunner.class) public class MultiLangDaemonConfigTest { - private static final String FILENAME = "some.properties"; + private static final String FILENAME = "multilang.properties"; private static final String EXE = "TestExe.exe"; private static final String APPLICATION_NAME = MultiLangDaemonConfigTest.class.getSimpleName(); private static final String STREAM_NAME = "fakeStream"; @@ -52,7 +52,7 @@ public class MultiLangDaemonConfigTest { @Mock private AwsCredentials creds; - private KinesisClientLibConfigurator configurator; + private final KinesisClientLibConfigurator configurator = new KinesisClientLibConfigurator(); private MultiLangDaemonConfig deamonConfig; /** @@ -62,7 +62,6 @@ public class MultiLangDaemonConfigTest { * @throws IOException */ public void setup(String streamName, String streamArn) throws IOException { - String properties = String.format("executableName = %s\n" + "applicationName = %s\n" + "AWSCredentialsProvider = DefaultAWSCredentialsProviderChain\n" @@ -85,8 +84,6 @@ public class MultiLangDaemonConfigTest { when(credentialsProvider.resolveCredentials()).thenReturn(creds); when(creds.accessKeyId()).thenReturn("cool-user"); - configurator = new KinesisClientLibConfigurator(); - deamonConfig = new MultiLangDaemonConfig(FILENAME, classLoader, configurator); } @@ -201,4 +198,14 @@ public class MultiLangDaemonConfigTest { } } + /** + * Test the loading of a "real" properties file. This test should catch + * any issues which might arise if there is a discrepancy between reality + * and mocking. + */ + @Test + public void testActualPropertiesFile() throws Exception { + new MultiLangDaemonConfig(FILENAME); + } + } \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/NestedPropertyKeyTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/NestedPropertyKeyTest.java new file mode 100644 index 00000000..3f61db7a --- /dev/null +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/NestedPropertyKeyTest.java @@ -0,0 +1,112 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.multilang; + +import com.amazonaws.regions.Regions; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static software.amazon.kinesis.multilang.NestedPropertyKey.ENDPOINT; +import static software.amazon.kinesis.multilang.NestedPropertyKey.ENDPOINT_REGION; +import static software.amazon.kinesis.multilang.NestedPropertyKey.EXTERNAL_ID; +import static software.amazon.kinesis.multilang.NestedPropertyKey.parse; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class NestedPropertyKeyTest { + + @Mock + private NestedPropertyProcessor mockProcessor; + + @Test + public void testExternalId() { + final String expectedId = "eid"; + + parse(mockProcessor, createKey(EXTERNAL_ID, expectedId)); + verify(mockProcessor).acceptExternalId(expectedId); + } + + @Test + public void testEndpoint() { + final String expectedEndpoint = "https://sts.us-east-1.amazonaws.com"; + final String expectedRegion = "us-east-1"; + final String param = createKey(ENDPOINT, expectedEndpoint + "^" + expectedRegion); + + parse(mockProcessor, param); + verify(mockProcessor).acceptEndpoint(expectedEndpoint, expectedRegion); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidEndpoint() { + parse(mockProcessor, createKey(ENDPOINT, "value-sans-caret-delimiter")); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidEndpointDoubleCaret() { + parse(mockProcessor, createKey(ENDPOINT, "https://sts.us-east-1.amazonaws.com^us-east-1^borkbork")); + } + + @Test + public void testEndpointRegion() { + final Regions expectedRegion = Regions.GovCloud; + + parse(mockProcessor, createKey(ENDPOINT_REGION, expectedRegion.getName())); + verify(mockProcessor).acceptEndpointRegion(expectedRegion); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidEndpointRegion() { + parse(mockProcessor, createKey(ENDPOINT_REGION, "snuffleupagus")); + } + + /** + * Test that the literal nested key (i.e., {@code key=} in {@code some_val|key=nested_val}) + * does not change. Any change to an existing literal key is not backwards-compatible. + */ + @Test + public void testKeysExplicitly() { + // Adding a new enum will deliberately cause this assert to fail, and + // therefore raise awareness for this explicit test. Add-and-remove may + // keep the number unchanged yet will also break (by removing an enum). + assertEquals(3, NestedPropertyKey.values().length); + + assertEquals("endpoint", ENDPOINT.getNestedKey()); + assertEquals("endpointRegion", ENDPOINT_REGION.getNestedKey()); + assertEquals("externalId", EXTERNAL_ID.getNestedKey()); + } + + @Test + public void testNonmatchingParameters() { + final String[] params = new String[] { + null, + "", + "hello world", // no nested key + "foo=bar", // nested key, but is not a recognized key + createKey(EXTERNAL_ID, "eid") + "=extra", // valid key made invalid by second '=' + }; + parse(mockProcessor, params); + verifyZeroInteractions(mockProcessor); + } + + private static String createKey(final NestedPropertyKey key, final String value) { + return key.getNestedKey() + "=" + value; + } + +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProviderTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProviderTest.java new file mode 100644 index 00000000..1c9e6bca --- /dev/null +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProviderTest.java @@ -0,0 +1,71 @@ +/* + * Copyright 2023 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.multilang.auth; + +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class KclSTSAssumeRoleSessionCredentialsProviderTest { + + private static final String ARN = "arn"; + private static final String SESSION_NAME = "sessionName"; + + /** + * Test that the constructor doesn't throw an out-of-bounds exception if + * there are no parameters beyond the required ARN and session name. + */ + @Test + public void testConstructorWithoutOptionalParams() { + new KclSTSAssumeRoleSessionCredentialsProvider(new String[] { ARN, SESSION_NAME }); + } + + @Test + public void testAcceptEndpoint() { + // discovered exception during e2e testing; therefore, this test is + // to simply verify the constructed STS client doesn't go *boom* + final KclSTSAssumeRoleSessionCredentialsProvider provider = + new KclSTSAssumeRoleSessionCredentialsProvider(ARN, SESSION_NAME); + provider.acceptEndpoint("endpoint", "us-east-1"); + } + + @Test + public void testVarArgs() { + for (final String[] varargs : Arrays.asList( + new String[] { ARN, SESSION_NAME, "externalId=eid", "foo"}, + new String[] { ARN, SESSION_NAME, "foo", "externalId=eid"} + )) { + final VarArgsSpy provider = new VarArgsSpy(varargs); + assertEquals("eid", provider.externalId); + } + } + + private static class VarArgsSpy extends KclSTSAssumeRoleSessionCredentialsProvider { + + private String externalId; + + public VarArgsSpy(String[] args) { + super(args); + } + + @Override + public void acceptExternalId(final String externalId) { + this.externalId = externalId; + super.acceptExternalId(externalId); + } + } +} \ No newline at end of file diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java index ced63f24..80e67d26 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/AWSCredentialsProviderPropertyValueDecoderTest.java @@ -16,6 +16,8 @@ package software.amazon.kinesis.multilang.config; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import java.util.Arrays; @@ -26,6 +28,7 @@ import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; import org.junit.Test; +import software.amazon.kinesis.multilang.auth.KclSTSAssumeRoleSessionCredentialsProvider; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; @@ -38,7 +41,7 @@ public class AWSCredentialsProviderPropertyValueDecoderTest { private final String credentialName1 = AlwaysSucceedCredentialsProvider.class.getName(); private final String credentialName2 = ConstructorCredentialsProvider.class.getName(); - private AWSCredentialsProviderPropertyValueDecoder decoder = new AWSCredentialsProviderPropertyValueDecoder(); + private final AWSCredentialsProviderPropertyValueDecoder decoder = new AWSCredentialsProviderPropertyValueDecoder(); @ToString private static class AWSCredentialsMatcher extends TypeSafeDiagnosingMatcher { @@ -53,10 +56,6 @@ public class AWSCredentialsProviderPropertyValueDecoderTest { this.classMatcher = instanceOf(AWSCredentialsProviderChain.class); } - private AWSCredentialsMatcher(AWSCredentials expected) { - this(expected.getAWSAccessKeyId(), expected.getAWSSecretKey()); - } - @Override protected boolean matchesSafely(AWSCredentialsProvider item, Description mismatchDescription) { AWSCredentials actual = item.getCredentials(); @@ -114,6 +113,33 @@ public class AWSCredentialsProviderPropertyValueDecoderTest { assertThat(provider, hasCredentials("arg1", "arg2")); } + /** + * Test that providers in the multi-lang auth package can be resolved and instantiated. + */ + @Test + public void testKclAuthProvider() { + for (final String className : Arrays.asList( + KclSTSAssumeRoleSessionCredentialsProvider.class.getName(), // fully-qualified name + KclSTSAssumeRoleSessionCredentialsProvider.class.getSimpleName() // name-only; needs prefix + )) { + final AWSCredentialsProvider provider = decoder.decodeValue(className + "|arn|sessionName"); + assertNotNull(className, provider); + } + } + + /** + * Test that a provider can be instantiated by its varargs constructor. + */ + @Test + public void testVarArgAuthProvider() { + final String[] args = new String[] { "arg1", "arg2", "arg3" }; + final String className = VarArgCredentialsProvider.class.getName(); + final String encodedValue = className + "|" + String.join("|", args); + + final AWSCredentialsProvider provider = decoder.decodeValue(encodedValue); + assertEquals(Arrays.toString(args), provider.getCredentials().getAWSAccessKeyId()); + } + /** * This credentials provider will always succeed */ @@ -138,9 +164,9 @@ public class AWSCredentialsProviderPropertyValueDecoderTest { private String arg1; private String arg2; + @SuppressWarnings("unused") public ConstructorCredentialsProvider(String arg1) { - this.arg1 = arg1; - this.arg2 = "blank"; + this(arg1, "blank"); } public ConstructorCredentialsProvider(String arg1, String arg2) { @@ -158,4 +184,25 @@ public class AWSCredentialsProviderPropertyValueDecoderTest { } } + + private static class VarArgCredentialsProvider implements AWSCredentialsProvider { + + private final String[] args; + + public VarArgCredentialsProvider(final String[] args) { + this.args = args; + } + + @Override + public AWSCredentials getCredentials() { + // KISS solution to surface the constructor args + final String flattenedArgs = Arrays.toString(args); + return new BasicAWSCredentials(flattenedArgs, flattenedArgs); + } + + @Override + public void refresh() { + + } + } } diff --git a/amazon-kinesis-client-multilang/src/test/resources/multilang.properties b/amazon-kinesis-client-multilang/src/test/resources/multilang.properties new file mode 100644 index 00000000..34cb0c1a --- /dev/null +++ b/amazon-kinesis-client-multilang/src/test/resources/multilang.properties @@ -0,0 +1,93 @@ +# The script that abides by the multi-language protocol. This script will +# be executed by the MultiLangDaemon, which will communicate with this script +# over STDIN and STDOUT according to the multi-language protocol. +executableName = sample_kclpy_app.py + +# The Stream arn: arn:aws:kinesis:::stream/ +# Important: streamArn takes precedence over streamName if both are set +streamArn = arn:aws:kinesis:us-east-5:000000000000:stream/kclpysample + +# The name of an Amazon Kinesis stream to process. +# Important: streamArn takes precedence over streamName if both are set +streamName = kclpysample + +# Used by the KCL as the name of this application. Will be used as the name +# of an Amazon DynamoDB table which will store the lease and checkpoint +# information for workers with this application name +applicationName = MultiLangTest + +# Users can change the credentials provider the KCL will use to retrieve credentials. +# The DefaultAWSCredentialsProviderChain checks several other providers, which is +# described here: +# http://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/DefaultAWSCredentialsProviderChain.html +AWSCredentialsProvider = DefaultAWSCredentialsProviderChain + +# Appended to the user agent of the KCL. Does not impact the functionality of the +# KCL in any other way. +processingLanguage = python/3.8 + +# Valid options at TRIM_HORIZON or LATEST. +# See http://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetShardIterator.html#API_GetShardIterator_RequestSyntax +initialPositionInStream = TRIM_HORIZON + +# To specify an initial timestamp from which to start processing records, please specify timestamp value for 'initiatPositionInStreamExtended', +# and uncomment below line with right timestamp value. +# See more from 'Timestamp' under http://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetShardIterator.html#API_GetShardIterator_RequestSyntax +#initialPositionInStreamExtended = 1636609142 + +# The following properties are also available for configuring the KCL Worker that is created +# by the MultiLangDaemon. + +# The KCL defaults to us-east-1 +regionName = us-east-1 + +# Fail over time in milliseconds. A worker which does not renew it's lease within this time interval +# will be regarded as having problems and it's shards will be assigned to other workers. +# For applications that have a large number of shards, this msy be set to a higher number to reduce +# the number of DynamoDB IOPS required for tracking leases +failoverTimeMillis = 10000 + +# A worker id that uniquely identifies this worker among all workers using the same applicationName +# If this isn't provided a MultiLangDaemon instance will assign a unique workerId to itself. +workerId = "workerId" + +# Shard sync interval in milliseconds - e.g. wait for this long between shard sync tasks. +shardSyncIntervalMillis = 60000 + +# Max records to fetch from Kinesis in a single GetRecords call. +maxRecords = 10000 + +# Idle time between record reads in milliseconds. +idleTimeBetweenReadsInMillis = 1000 + +# Enables applications flush/checkpoint (if they have some data "in progress", but don't get new data for while) +callProcessRecordsEvenForEmptyRecordList = false + +# Interval in milliseconds between polling to check for parent shard completion. +# Polling frequently will take up more DynamoDB IOPS (when there are leases for shards waiting on +# completion of parent shards). +parentShardPollIntervalMillis = 10000 + +# Cleanup leases upon shards completion (don't wait until they expire in Kinesis). +# Keeping leases takes some tracking/resources (e.g. they need to be renewed, assigned), so by default we try +# to delete the ones we don't need any longer. +cleanupLeasesUponShardCompletion = true + +# Backoff time in milliseconds for Amazon Kinesis Client Library tasks (in the event of failures). +taskBackoffTimeMillis = 500 + +# Buffer metrics for at most this long before publishing to CloudWatch. +metricsBufferTimeMillis = 10000 + +# Buffer at most this many metrics before publishing to CloudWatch. +metricsMaxQueueSize = 10000 + +# KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls +# to RecordProcessorCheckpointer#checkpoint(String) by default. +validateSequenceNumberBeforeCheckpointing = true + +# The maximum number of active threads for the MultiLangDaemon to permit. +# If a value is provided then a FixedThreadPool is used with the maximum +# active threads set to the provided value. If a non-positive integer or no +# value is provided a CachedThreadPool is used. +maxActiveThreads = -1 From 12b9a36d0db1f99670086be129c7f86f5b34a3b5 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 7 Aug 2023 16:56:25 -0400 Subject: [PATCH 86/88] Provided documentation for multilang's new NestedPropertyKey enhancement. (#1186) --- .../configuring-credential-providers.md | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 docs/multilang/configuring-credential-providers.md diff --git a/docs/multilang/configuring-credential-providers.md b/docs/multilang/configuring-credential-providers.md new file mode 100644 index 00000000..9b85baaa --- /dev/null +++ b/docs/multilang/configuring-credential-providers.md @@ -0,0 +1,71 @@ +# Configuring Credential Providers + +[AwsCredentialProviders][aws-credentials-provider] are not a one-size-fits-all. +The AWS SDK provides a rich API to support various configurations for many different providers. +KCL multilang does not, and is not intended to, proxy the full breadth of the AWS SDK. +However, KCL now provides better extensibility to handle, and be enhanced to handle, additional configurations. +This document should help multilang customers configure a suitable `CredentialProvider` (or contribute changes to support a new use case!). + +## Sample Provider Configuration + +In a Properties file, an `AWSCredentialsProperty` configuration might look like: +``` +AWSCredentialsProvider = STSAssumeRoleSessionCredentialsProvider|| +``` +This basic configuration creates an [STSAssumeRoleSessionCredentialsProvider][sts-assume-provider] with an ARN and session name. +While functional, this configuration is limited. +For example, this configuration cannot set a regional endpoint (e.g., VPC use case). + +Leveraging nested properties, an `AWSCredentialsProperty` value might change to: +``` +AWSCredentialsProvider = KclSTSAssumeRoleSessionCredentialsProvider||\ + |endpointRegion=us-east-1|externalId=spartacus +``` +N.B. Backslash (`\`) is for multi-line legibility and is not required. + +## Nested Properties + +KCL multilang supports "nested properties" on the `AWSCredentialsProvider` key in the properties file. +The [Backus-Naur form][bnf] of the value: +``` + ::= ["|" ]* ["|" ]* + ::= | + ::= # this depends on the provider + ::= "=" + ::= + # this depends on the nested key +``` + +In general, required parameters are passed directly to the class' constructor +(e.g., [STSAssumeRoleSessionCredentialsProvider(String, String)][sts-assume-provider-constructor]). + +Nested properties are a custom mapping provided by KCL multilang, and do not exist in the AWS SDK. +See [NestedPropertyKey][nested-property-key] for the supported keys, and details on their expected values. + +## Nested Property Processor + +Nested keys are processed via [NestedPropertyProcessor][nested-property-processor]. +Implementation is, obviously, dependent on the implementing class. +Adding a new nested key should be trivial. +A backwards-compatible addition might look like: +``` + default void acceptFoo(...) { + // do nothing + } +``` + +### KclSTSAssumeRoleSessionCredentialsProvider + +KCL multilang includes a [custom nested property processor for `STSAssumeRole`][kcl-sts-provider]. +Multilang configurations that use `STSAssumeRoleSessionCredentialsProvider` need only prefix `Kcl` to exercise this new provider: +``` +AWSCredentialsProvider = KclSTSAssumeRoleSessionCredentialsProvider|| +``` + +[aws-credentials-provider]: https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/auth/credentials/AwsCredentialsProvider.html +[bnf]: https://en.wikipedia.org/wiki/Backus%E2%80%93Naur_form +[kcl-sts-provider]: /amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/auth/KclSTSAssumeRoleSessionCredentialsProvider.java +[nested-property-key]: /amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyKey.java +[nested-property-processor]: /amazon-kinesis-client-multilang/src/main/java/software/amazon/kinesis/multilang/NestedPropertyProcessor.java +[sts-assume-provider]: https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/STSAssumeRoleSessionCredentialsProvider.html +[sts-assume-provider-constructor]: https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/auth/STSAssumeRoleSessionCredentialsProvider.html#STSAssumeRoleSessionCredentialsProvider-java.lang.String-java.lang.String- From 7384bc1dbecb36bcb6032b590cd0052c75c1c1e4 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 7 Aug 2023 20:54:53 -0400 Subject: [PATCH 87/88] Release of 2.5.2 (#1187) --- CHANGELOG.md | 12 ++++++++++++ amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 5 files changed, 16 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fc7ec987..f9497e1c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,18 @@ For **1.x** release notes, please see [v1.x/CHANGELOG.md](https://github.com/awslabs/amazon-kinesis-client/blob/v1.x/CHANGELOG.md) --- +### Release 2.5.2 (2023-08-07) +* [#1184](https://github.com/awslabs/amazon-kinesis-client/pull/1184) [#367] Enhanced multi-lang `AWSCredentialsProvider=...` decoder and c… +* [#1186](https://github.com/awslabs/amazon-kinesis-client/pull/1186) Provided documentation for multilang's new NestedPropertyKey enhancement. +* [#1181](https://github.com/awslabs/amazon-kinesis-client/pull/1181) CVE-2023-2976: Upgrade Google Guava dependency version from `32.0.0-jre` to `32.1.1-jre` +* [#1159](https://github.com/awslabs/amazon-kinesis-client/pull/1159) Bug fix in lease refresher integration test with occasional failures +* [#1157](https://github.com/awslabs/amazon-kinesis-client/pull/1157) Fix NPE on graceful shutdown before DDB `LeaseCoordinator` starts. +* [#1152](https://github.com/awslabs/amazon-kinesis-client/pull/1152) Adding resharding integration tests and changing ITs to not run by default +* [#1162](https://github.com/awslabs/amazon-kinesis-client/pull/1162) Only deleting resource created by ITs +* [#1158](https://github.com/awslabs/amazon-kinesis-client/pull/1158) Checkstyle: tightened `LineLength` restriction from 170 to 150. +* [#1151](https://github.com/awslabs/amazon-kinesis-client/pull/1151) Modified `dependabot.yml` to set the correct `v[1|2].x` label. +* [#1164](https://github.com/awslabs/amazon-kinesis-client/pull/1164) Upgraded KCL Version from 2.5.1 to 2.5.2-SNAPSHOT + ### Release 2.5.1 (June 27, 2023) * [#1143](https://github.com/awslabs/amazon-kinesis-client/pull/1143) Upgrade MultiLangDaemon to support StreamARN * [#1145](https://github.com/awslabs/amazon-kinesis-client/pull/1145) Introduced GitHub actions to trigger Maven builds during merge/pull requests diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 673b3eaa..7ed461dd 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.5.2-SNAPSHOT + 2.5.2 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 653d581f..4f69ffee 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.5.2-SNAPSHOT + 2.5.2 amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index ba9f791f..032a4eaf 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.2-SNAPSHOT"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.2"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index abd1308e..2026ba40 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.5.2-SNAPSHOT + 2.5.2 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From a1731dc49b2d76b8dd33781d5cabc5264b7079f9 Mon Sep 17 00:00:00 2001 From: stair <123031771+stair-aws@users.noreply.github.com> Date: Mon, 7 Aug 2023 21:13:07 -0400 Subject: [PATCH 88/88] Reintroduced `-SNAPSHOT` classifier on KCL version. (#1188) --- amazon-kinesis-client-multilang/pom.xml | 2 +- amazon-kinesis-client/pom.xml | 2 +- .../java/software/amazon/kinesis/retrieval/RetrievalConfig.java | 2 +- pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/amazon-kinesis-client-multilang/pom.xml b/amazon-kinesis-client-multilang/pom.xml index 7ed461dd..bafa2bfa 100644 --- a/amazon-kinesis-client-multilang/pom.xml +++ b/amazon-kinesis-client-multilang/pom.xml @@ -21,7 +21,7 @@ amazon-kinesis-client-pom software.amazon.kinesis - 2.5.2 + 2.5.3-SNAPSHOT 4.0.0 diff --git a/amazon-kinesis-client/pom.xml b/amazon-kinesis-client/pom.xml index 4f69ffee..7f7f5f4c 100644 --- a/amazon-kinesis-client/pom.xml +++ b/amazon-kinesis-client/pom.xml @@ -22,7 +22,7 @@ software.amazon.kinesis amazon-kinesis-client-pom - 2.5.2 + 2.5.3-SNAPSHOT amazon-kinesis-client diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 032a4eaf..d20a57c1 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -49,7 +49,7 @@ public class RetrievalConfig { */ public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java"; - public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.2"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT_VERSION = "2.5.3-SNAPSHOT"; /** * Client used to make calls to Kinesis for records retrieval diff --git a/pom.xml b/pom.xml index 2026ba40..8844aa53 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ amazon-kinesis-client-pom pom Amazon Kinesis Client Library - 2.5.2 + 2.5.3-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis.