From 62e13ff3a18e6407a59e7000733a544dfb9fe022 Mon Sep 17 00:00:00 2001 From: Jegosh John Date: Tue, 25 Feb 2020 10:41:34 -0800 Subject: [PATCH 001/159] Introduce PeriodicShardSync with leader election --- ...ministicShuffleShardSyncLeaderDecider.java | 152 ++++++++++++++++++ .../kinesis/coordinator/LeaderDecider.java | 39 +++++ .../coordinator/PeriodicShardSyncManager.java | 94 +++++++++++ .../amazon/kinesis/coordinator/Scheduler.java | 49 ++++-- .../software/amazon/kinesis/leases/Lease.java | 14 +- 5 files changed, 324 insertions(+), 24 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java new file mode 100644 index 00000000..720103c6 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDecider.java @@ -0,0 +1,152 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; + +import java.time.Instant; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.BooleanSupplier; +import java.util.stream.Collectors; + +/** + * An implementation of the {@code LeaderDecider} to elect leader(s) based on workerId. + * Leases are shuffled using a predetermined constant seed so that lease ordering is + * preserved across workers. + * This reduces the probability of choosing the leader workers co-located on the same + * host in case workerId starts with a common string (e.g. IP Address). + * Hence if a host has 3 workers, IPADDRESS_Worker1, IPADDRESS_Worker2, and IPADDRESS_Worker3, + * we don't end up choosing all 3 for shard sync as a result of natural ordering of Strings. + * This ensures redundancy for shard-sync during host failures. + */ +@Slf4j +class DeterministicShuffleShardSyncLeaderDecider + implements LeaderDecider { + // Fixed seed so that the shuffle order is preserved across workers + static final int DETERMINISTIC_SHUFFLE_SEED = 1947; + + private static final long ELECTION_INITIAL_DELAY_MILLIS = 60 * 1000; + private static final long ELECTION_SCHEDULING_INTERVAL_MILLIS = 5 * 60 * 1000; + private static final int AWAIT_TERMINATION_MILLIS = 5000; + + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + + private final LeaseRefresher leaseRefresher; + private final int numPeriodicShardSyncWorkers; + private final ScheduledExecutorService leaderElectionThreadPool; + + private volatile Set leaders; + + /** + * @param leaseRefresher LeaseManager instance used to fetch leases. + * @param leaderElectionThreadPool Thread-pool to be used for leaderElection. + * @param numPeriodicShardSyncWorkers Number of leaders that will be elected to perform periodic shard syncs. + */ + DeterministicShuffleShardSyncLeaderDecider(LeaseRefresher leaseRefresher, ScheduledExecutorService leaderElectionThreadPool, + int numPeriodicShardSyncWorkers) { + this.leaseRefresher = leaseRefresher; + this.leaderElectionThreadPool = leaderElectionThreadPool; + this.numPeriodicShardSyncWorkers = numPeriodicShardSyncWorkers; + } + + /* + * Shuffles the leases deterministically and elects numPeriodicShardSyncWorkers number of workers + * as leaders (workers that will perform shard sync). + */ + private void electLeaders() { + try { + log.debug("Started leader election at: " + Instant.now()); + List leases = leaseRefresher.listLeases(); + List uniqueHosts = leases.stream().map(Lease::leaseOwner) + .filter(owner -> owner != null).distinct().sorted().collect(Collectors.toList()); + + Collections.shuffle(uniqueHosts, new Random(DETERMINISTIC_SHUFFLE_SEED)); + int numShardSyncWorkers = Math.min(uniqueHosts.size(), numPeriodicShardSyncWorkers); + // In case value is currently being read, we wait for reading to complete before updating the variable. + // This is to prevent any ConcurrentModificationException exceptions. + readWriteLock.writeLock().lock(); + leaders = new HashSet<>(uniqueHosts.subList(0, numShardSyncWorkers)); + log.info("Elected leaders: " + String.join(", ", leaders)); + log.debug("Completed leader election at: " + Instant.now()); + } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { + log.error("Exception occurred while trying to fetch all leases for leader election", e); + } catch (Throwable t) { + log.error("Unknown exception during leader election.", t); + } finally { + readWriteLock.writeLock().unlock(); + } + } + + private boolean isWorkerLeaderForShardSync(String workerId) { + return CollectionUtils.isNullOrEmpty(leaders) || leaders.contains(workerId); + } + + @Override + public synchronized Boolean isLeader(String workerId) { + // if no leaders yet, synchronously get leaders. This will happen at first Shard Sync. + if (executeConditionCheckWithReadLock(() -> CollectionUtils.isNullOrEmpty(leaders))) { + electLeaders(); + // start a scheduled executor that will periodically update leaders. + // The first run will be after a minute. + // We don't need jitter since it is scheduled with a fixed delay and time taken to scan leases + // will be different at different times and on different hosts/workers. + leaderElectionThreadPool.scheduleWithFixedDelay(this::electLeaders, ELECTION_INITIAL_DELAY_MILLIS, + ELECTION_SCHEDULING_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); + } + + return executeConditionCheckWithReadLock(() -> isWorkerLeaderForShardSync(workerId)); + } + + @Override + public synchronized void shutdown() { + try { + leaderElectionThreadPool.shutdown(); + if (leaderElectionThreadPool.awaitTermination(AWAIT_TERMINATION_MILLIS, TimeUnit.MILLISECONDS)) { + log.info("Successfully stopped leader election on the worker"); + } else { + leaderElectionThreadPool.shutdownNow(); + log.info(String.format("Stopped leader election thread after awaiting termination for %d milliseconds", + AWAIT_TERMINATION_MILLIS)); + } + + } catch (InterruptedException e) { + log.debug("Encountered InterruptedException while awaiting leader election threadPool termination"); + } + } + + // Execute condition checks using shared variables under a read-write lock. + private boolean executeConditionCheckWithReadLock(BooleanSupplier action) { + try { + readWriteLock.readLock().lock(); + return action.getAsBoolean(); + } finally { + readWriteLock.readLock().unlock(); + } + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java new file mode 100644 index 00000000..140791af --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/LeaderDecider.java @@ -0,0 +1,39 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +/** + * Used in conjunction with periodic shard sync. + * Implement this interface to allow KCL to decide if the current worker should execute shard sync. + * When periodic shard sync is enabled, PeriodicShardSyncManager periodically checks if the current + * worker is one of the leaders designated to execute shard-sync and then acts accordingly. + */ +public interface LeaderDecider { + + /** + * Method invoked to check the given workerId corresponds to one of the workers + * designated to execute shard-syncs periodically. + * + * @param workerId ID of the worker + * @return True if the worker with ID workerId can execute shard-sync. False otherwise. + */ + Boolean isLeader(String workerId); + + /** + * Can be invoked, if needed, to shutdown any clients/thread-pools + * being used in the LeaderDecider implementation. + */ + void shutdown(); +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java new file mode 100644 index 00000000..3134fef7 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManager.java @@ -0,0 +1,94 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.Validate; +import software.amazon.kinesis.leases.ShardSyncTask; +import software.amazon.kinesis.lifecycle.ConsumerTask; +import software.amazon.kinesis.lifecycle.TaskResult; +import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; +import software.amazon.kinesis.metrics.MetricsFactory; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * The top level orchestrator for coordinating the periodic shard sync related + * activities. + */ +@Getter +@EqualsAndHashCode +@Slf4j +class PeriodicShardSyncManager { + private static final long INITIAL_DELAY = 0; + private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 5 * 60 * 1000; + + private final String workerId; + private final LeaderDecider leaderDecider; + private final ConsumerTask metricsEmittingShardSyncTask; + private final ScheduledExecutorService shardSyncThreadPool; + private boolean isRunning; + + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, MetricsFactory metricsFactory) { + this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory); + } + + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, ScheduledExecutorService shardSyncThreadPool, MetricsFactory metricsFactory) { + Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); + Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); + Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager."); + this.workerId = workerId; + this.leaderDecider = leaderDecider; + this.metricsEmittingShardSyncTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory); + this.shardSyncThreadPool = shardSyncThreadPool; + } + + public synchronized TaskResult start() { + if (!isRunning) { + shardSyncThreadPool + .scheduleWithFixedDelay(this::runShardSync, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, + TimeUnit.MILLISECONDS); + isRunning = true; + } + return new TaskResult(null); + } + + public void stop() { + if (isRunning) { + log.info(String.format("Shutting down leader decider on worker %s", workerId)); + leaderDecider.shutdown(); + log.info(String.format("Shutting down periodic shard sync task scheduler on worker %s", workerId)); + shardSyncThreadPool.shutdown(); + isRunning = false; + } + } + + private void runShardSync() { + try { + if (leaderDecider.isLeader(workerId)) { + log.info(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); + metricsEmittingShardSyncTask.call(); + } else { + log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); + } + } catch (Throwable t) { + log.error("Error during runShardSync.", t); + } + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index eaeb5a1c..7f92c384 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -15,20 +15,7 @@ package software.amazon.kinesis.coordinator; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - import com.google.common.annotations.VisibleForTesting; - import io.reactivex.plugins.RxJavaPlugins; import lombok.AccessLevel; import lombok.Getter; @@ -39,6 +26,7 @@ import lombok.extern.slf4j.Slf4j; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -48,7 +36,6 @@ import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardPrioritization; import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.leases.ShardSyncTaskManager; -import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; import software.amazon.kinesis.leases.exceptions.LeasingException; import software.amazon.kinesis.lifecycle.LifecycleConfig; @@ -70,6 +57,19 @@ import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + /** * */ @@ -78,6 +78,7 @@ import software.amazon.kinesis.retrieval.RetrievalConfig; @Slf4j public class Scheduler implements Runnable { + private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1; private SchedulerLog slog = new SchedulerLog(); private final CheckpointConfig checkpointConfig; @@ -101,6 +102,7 @@ public class Scheduler implements Runnable { // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; private final ShardSyncTaskManager shardSyncTaskManager; + private final PeriodicShardSyncManager periodicShardSyncManager; private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; @@ -119,6 +121,7 @@ public class Scheduler implements Runnable { private final AggregatorUtil aggregatorUtil; private final HierarchicalShardSyncer hierarchicalShardSyncer; private final long schedulerInitializationBackoffTimeMillis; + private LeaderDecider leaderDecider; // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. @@ -209,6 +212,10 @@ public class Scheduler implements Runnable { this.workerStateChangeListener = this.coordinatorConfig.coordinatorFactory() .createWorkerStateChangeListener(); } + if (leaderDecider == null) { + leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, + Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); + } this.initialPosition = retrievalConfig.initialPositionInStreamExtended(); this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis(); this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis(); @@ -222,6 +229,11 @@ public class Scheduler implements Runnable { this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); + ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, + cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, + metricsFactory); + this.periodicShardSyncManager = new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), + leaderDecider, shardSyncTask, metricsFactory); } /** @@ -235,13 +247,17 @@ public class Scheduler implements Runnable { try { initialize(); - log.info("Initialization complete. Starting worker loop."); + log.info("Initialization complete. Scheduling periodicShardSync.."); + + periodicShardSyncManager.start(); + + log.info("Scheduled periodicShardSync tasks. Starting worker loop."); + } catch (RuntimeException e) { log.error("Unable to initialize after {} attempts. Shutting down.", maxInitializationAttempts, e); workerStateChangeListener.onAllInitializationAttemptsFailed(e); shutdown(); } - while (!shouldShutdown()) { runProcessLoop(); } @@ -516,6 +532,7 @@ public class Scheduler implements Runnable { // Lost leases will force Worker to begin shutdown process for all shard consumers in // Worker.run(). leaseCoordinator.stop(); + periodicShardSyncManager.stop(); workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.SHUT_DOWN); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java index 802ee29b..682a6f9e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java @@ -14,14 +14,6 @@ */ package software.amazon.kinesis.leases; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.TimeUnit; - -import com.google.common.collect.Collections2; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; @@ -30,6 +22,12 @@ import lombok.ToString; import lombok.experimental.Accessors; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + /** * This class contains data pertaining to a Lease. Distributed systems may use leases to partition work across a * fleet of workers. Each unit of work (identified by a leaseKey) has a corresponding Lease. Every worker will contend From 8511475868ec7bf502afa9396ee2fb4dcb29ce94 Mon Sep 17 00:00:00 2001 From: Jegosh John Date: Tue, 3 Mar 2020 16:02:00 -0800 Subject: [PATCH 002/159] Minor refactor, addressing feedback --- .../coordinator/PeriodicShardSyncManager.java | 55 ++++++-- .../amazon/kinesis/coordinator/Scheduler.java | 42 +++--- ...sticShuffleShardSyncLeaderDeciderTest.java | 125 ++++++++++++++++++ 3 files changed, 188 insertions(+), 34 deletions(-) create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java 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 3134fef7..7b10694f 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 @@ -18,6 +18,7 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; +import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.lifecycle.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; @@ -36,8 +37,8 @@ import java.util.concurrent.TimeUnit; @EqualsAndHashCode @Slf4j class PeriodicShardSyncManager { - private static final long INITIAL_DELAY = 0; - private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 5 * 60 * 1000; + private static final long INITIAL_DELAY = 60 * 1000L; + private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 5 * 60 * 1000L; private final String workerId; private final LeaderDecider leaderDecider; @@ -61,14 +62,34 @@ class PeriodicShardSyncManager { public synchronized TaskResult start() { if (!isRunning) { - shardSyncThreadPool - .scheduleWithFixedDelay(this::runShardSync, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, - TimeUnit.MILLISECONDS); + final Runnable periodicShardSyncer = () -> { + try { + runShardSync(); + } catch (Throwable t) { + log.error("Error during runShardSync.", t); + } + }; + shardSyncThreadPool.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, + TimeUnit.MILLISECONDS); isRunning = true; + } return new TaskResult(null); } + public synchronized TaskResult syncShardsOnce() { + + Exception lastException = null; + try { + if (!isRunning) { + runShardSync(); + } + } catch (Exception e) { + lastException = e; + } + return new TaskResult(lastException); + } + public void stop() { if (isRunning) { log.info(String.format("Shutting down leader decider on worker %s", workerId)); @@ -80,15 +101,23 @@ class PeriodicShardSyncManager { } private void runShardSync() { - try { - if (leaderDecider.isLeader(workerId)) { - log.info(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); - metricsEmittingShardSyncTask.call(); - } else { - log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); + if (leaderDecider.isLeader(workerId)) { + log.info(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); + final TaskResult taskResult = metricsEmittingShardSyncTask.call(); + if (taskResult != null && taskResult.getException() != null) { + throw new KinesisClientLibIOException("Failed to sync shards", taskResult.getException()); } - } catch (Throwable t) { - log.error("Error during runShardSync.", t); + } else { + log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); } } + + /** + * Checks if the entire hash range is covered + * @return true if covered, false otherwise + */ + public boolean hashRangeCovered() { + // TODO: Implement method + return true; + } } 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 7f92c384..87b92162 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 @@ -46,7 +46,6 @@ import software.amazon.kinesis.lifecycle.ShutdownNotification; import software.amazon.kinesis.lifecycle.ShutdownReason; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.CloudWatchMetricsFactory; -import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -68,6 +67,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; /** @@ -102,7 +102,7 @@ public class Scheduler implements Runnable { // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; private final ShardSyncTaskManager shardSyncTaskManager; - private final PeriodicShardSyncManager periodicShardSyncManager; + private final PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager; private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; @@ -121,7 +121,7 @@ public class Scheduler implements Runnable { private final AggregatorUtil aggregatorUtil; private final HierarchicalShardSyncer hierarchicalShardSyncer; private final long schedulerInitializationBackoffTimeMillis; - private LeaderDecider leaderDecider; + private final LeaderDecider leaderDecider; // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. @@ -212,10 +212,8 @@ public class Scheduler implements Runnable { this.workerStateChangeListener = this.coordinatorConfig.coordinatorFactory() .createWorkerStateChangeListener(); } - if (leaderDecider == null) { - leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, + this.leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); - } this.initialPosition = retrievalConfig.initialPositionInStreamExtended(); this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis(); this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis(); @@ -232,7 +230,7 @@ public class Scheduler implements Runnable { ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, metricsFactory); - this.periodicShardSyncManager = new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), + this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), leaderDecider, shardSyncTask, metricsFactory); } @@ -247,12 +245,7 @@ public class Scheduler implements Runnable { try { initialize(); - log.info("Initialization complete. Scheduling periodicShardSync.."); - - periodicShardSyncManager.start(); - - log.info("Scheduled periodicShardSync tasks. Starting worker loop."); - + log.info("Initialization complete. Starting worker loop."); } catch (RuntimeException e) { log.error("Unable to initialize after {} attempts. Shutting down.", maxInitializationAttempts, e); workerStateChangeListener.onAllInitializationAttemptsFailed(e); @@ -282,11 +275,14 @@ public class Scheduler implements Runnable { TaskResult result = null; if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { + for (int j = 0; j < 10 && leaseRefresher.isLeaseTableEmpty(); j++) { + // check every 1-5 seconds if lease table is still empty, + // to minimize contention between all workers bootstrapping at the same time + long waitTime = ThreadLocalRandom.current().nextLong(1000L, 5000L); + Thread.sleep(waitTime); + } log.info("Syncing Kinesis shard info"); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, - metricsFactory); - result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); + result = leaderElectedPeriodicShardSyncManager.syncShardsOnce(); } else { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); } @@ -298,6 +294,9 @@ public class Scheduler implements Runnable { } else { log.info("LeaseCoordinator is already running. No need to start it."); } + log.info("Scheduling periodicShardSync)"); + // leaderElectedPeriodicShardSyncManager.start(); + // TODO: enable periodicShardSync after https://github.com/jushkem/amazon-kinesis-client/pull/2 is merged isDone = true; } else { lastException = result.getException(); @@ -309,9 +308,10 @@ public class Scheduler implements Runnable { lastException = e; } - if (!isDone) { + if (!isDone || !leaderElectedPeriodicShardSyncManager.hashRangeCovered()) { try { Thread.sleep(schedulerInitializationBackoffTimeMillis); + leaderElectedPeriodicShardSyncManager.stop(); } catch (InterruptedException e) { log.debug("Sleep interrupted while initializing worker."); } @@ -532,7 +532,7 @@ public class Scheduler implements Runnable { // Lost leases will force Worker to begin shutdown process for all shard consumers in // Worker.run(). leaseCoordinator.stop(); - periodicShardSyncManager.stop(); + leaderElectedPeriodicShardSyncManager.stop(); workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.SHUT_DOWN); } } @@ -630,12 +630,12 @@ public class Scheduler implements Runnable { hierarchicalShardSyncer, metricsFactory); return new ShardConsumer(cache, executorService, shardInfo, lifecycleConfig.logWarningForTaskAfterMillis(), - argument, lifecycleConfig.taskExecutionListener(),lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); + argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); } /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. - * + *

* This method relies on ShardInfo.equals() method returning true for ShardInfo objects which may have been * instantiated with parentShardIds in a different order (and rest of the fields being the equal). For example * shardInfo1.equals(shardInfo2) should return true with shardInfo1 and shardInfo2 defined as follows. ShardInfo diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java new file mode 100644 index 00000000..b6ff3a0d --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/DeterministicShuffleShardSyncLeaderDeciderTest.java @@ -0,0 +1,125 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.coordinator; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.when; +import static software.amazon.kinesis.coordinator.DeterministicShuffleShardSyncLeaderDecider.DETERMINISTIC_SHUFFLE_SEED; + +@RunWith(MockitoJUnitRunner.class) +public class DeterministicShuffleShardSyncLeaderDeciderTest { + private static final String LEASE_KEY = "lease_key"; + private static final String LEASE_OWNER = "lease_owner"; + private static final String WORKER_ID = "worker-id"; + + private DeterministicShuffleShardSyncLeaderDecider leaderDecider; + + @Mock + private LeaseRefresher leaseRefresher; + + @Mock + private ScheduledExecutorService scheduledExecutorService; + + private int numShardSyncWorkers; + + @Before + public void setup() { + numShardSyncWorkers = 1; + leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers); + } + + @Test + public void testLeaderElectionWithNullLeases() { + boolean isLeader = leaderDecider.isLeader(WORKER_ID); + assertTrue("IsLeader should return true if leaders is null", isLeader); + } + + @Test + public void testLeaderElectionWithEmptyLeases() throws Exception { + when(leaseRefresher.listLeases()).thenReturn(new ArrayList<>()); + boolean isLeader = leaderDecider.isLeader(WORKER_ID); + assertTrue("IsLeader should return true if no leases are returned", isLeader); + } + + @Test + public void testElectedLeadersAsPerExpectedShufflingOrder() + throws Exception { + List leases = getLeases(5, false /* duplicateLeaseOwner */, true /* activeLeases */); + when(leaseRefresher.listLeases()).thenReturn(leases); + Set expectedLeaders = getExpectedLeaders(leases); + for (String leader : expectedLeaders) { + assertTrue(leaderDecider.isLeader(leader)); + } + for (Lease lease : leases) { + if (!expectedLeaders.contains(lease.leaseOwner())) { + assertFalse(leaderDecider.isLeader(lease.leaseOwner())); + } + } + } + + @Test + public void testElectedLeadersAsPerExpectedShufflingOrderWhenUniqueWorkersLessThanMaxLeaders() { + this.numShardSyncWorkers = 5; // More than number of unique lease owners + leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers); + List leases = getLeases(3, false /* duplicateLeaseOwner */, true /* activeLeases */); + Set expectedLeaders = getExpectedLeaders(leases); + // All lease owners should be present in expected leaders set, and they should all be leaders. + for (Lease lease : leases) { + assertTrue(leaderDecider.isLeader(lease.leaseOwner())); + assertTrue(expectedLeaders.contains(lease.leaseOwner())); + } + } + + private List getLeases(int count, boolean duplicateLeaseOwner, boolean activeLeases) { + List leases = new ArrayList<>(); + for (int i = 0; i < count; i++) { + Lease lease = new Lease(); + lease.leaseKey(LEASE_KEY + i); + lease.checkpoint(activeLeases ? ExtendedSequenceNumber.LATEST : ExtendedSequenceNumber.SHARD_END); + lease.leaseCounter(new Random().nextLong()); + lease.leaseOwner(LEASE_OWNER + (duplicateLeaseOwner ? "" : i)); + leases.add(lease); + } + return leases; + } + + private Set getExpectedLeaders(List leases) { + List uniqueHosts = leases.stream().filter(lease -> lease.leaseOwner() != null) + .map(Lease::leaseOwner).distinct().sorted().collect(Collectors.toList()); + + Collections.shuffle(uniqueHosts, new Random(DETERMINISTIC_SHUFFLE_SEED)); + int numWorkers = Math.min(uniqueHosts.size(), this.numShardSyncWorkers); + return new HashSet<>(uniqueHosts.subList(0, numWorkers)); + } +} From 3bd9b29a1312426ec0cce17643579c62192afb9b Mon Sep 17 00:00:00 2001 From: Jegosh John Date: Fri, 6 Mar 2020 10:53:35 -0800 Subject: [PATCH 003/159] Add wait to initialization until hashRange covered, modify wait logic for empty leaseTable check --- .../coordinator/PeriodicShardSyncManager.java | 5 ++ .../amazon/kinesis/coordinator/Scheduler.java | 54 ++++++++++++++----- .../kinesis/coordinator/SchedulerTest.java | 39 ++++++++++++++ 3 files changed, 86 insertions(+), 12 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 7b10694f..7d166b98 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 @@ -77,6 +77,11 @@ class PeriodicShardSyncManager { return new TaskResult(null); } + /** + * Runs shardSync once + * Does not schedule periodic shardSync + * @return the result of the task + */ public synchronized TaskResult syncShardsOnce() { Exception lastException = null; 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 87b92162..6f4d4ed1 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 @@ -37,7 +37,10 @@ import software.amazon.kinesis.leases.ShardPrioritization; import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.LeasingException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; import software.amazon.kinesis.lifecycle.ShardConsumerArgument; @@ -79,6 +82,10 @@ import java.util.concurrent.TimeUnit; 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 = 5 * 1000L; + private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; + private static final long HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS = 5000L; private SchedulerLog slog = new SchedulerLog(); private final CheckpointConfig checkpointConfig; @@ -227,11 +234,7 @@ public class Scheduler implements Runnable { this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, - metricsFactory); - this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), - leaderDecider, shardSyncTask, metricsFactory); + this.leaderElectedPeriodicShardSyncManager = buildPeriodicShardSyncManager(); } /** @@ -275,12 +278,7 @@ public class Scheduler implements Runnable { TaskResult result = null; if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { - for (int j = 0; j < 10 && leaseRefresher.isLeaseTableEmpty(); j++) { - // check every 1-5 seconds if lease table is still empty, - // to minimize contention between all workers bootstrapping at the same time - long waitTime = ThreadLocalRandom.current().nextLong(1000L, 5000L); - Thread.sleep(waitTime); - } + waitUntilLeaseTableIsReady(); log.info("Syncing Kinesis shard info"); result = leaderElectedPeriodicShardSyncManager.syncShardsOnce(); } else { @@ -297,6 +295,7 @@ public class Scheduler implements Runnable { log.info("Scheduling periodicShardSync)"); // leaderElectedPeriodicShardSyncManager.start(); // TODO: enable periodicShardSync after https://github.com/jushkem/amazon-kinesis-client/pull/2 is merged + waitUntilHashRangeCovered(); isDone = true; } else { lastException = result.getException(); @@ -308,7 +307,7 @@ public class Scheduler implements Runnable { lastException = e; } - if (!isDone || !leaderElectedPeriodicShardSyncManager.hashRangeCovered()) { + if (!isDone) { try { Thread.sleep(schedulerInitializationBackoffTimeMillis); leaderElectedPeriodicShardSyncManager.stop(); @@ -325,6 +324,29 @@ public class Scheduler implements Runnable { } } + @VisibleForTesting + void waitUntilLeaseTableIsReady() throws InterruptedException, + DependencyException, ProvisionedThroughputException, InvalidStateException { + long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + long waitUntil = System.currentTimeMillis() + waitTime; + + while (System.currentTimeMillis() < waitUntil && leaseRefresher.isLeaseTableEmpty()) { + // check every 3 seconds if lease table is still empty, + // to minimize contention between all workers bootstrapping at the same time + log.info("Lease table is still empty. Checking again in {} ms", LEASE_TABLE_CHECK_FREQUENCY_MILLIS); + Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS); + } + } + + private void waitUntilHashRangeCovered() throws InterruptedException { + + while (!leaderElectedPeriodicShardSyncManager.hashRangeCovered()) { + // wait until entire hash range is covered + log.info("Hash range is not covered yet. Checking again in {} ms", HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS); + Thread.sleep(HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS); + } + } + @VisibleForTesting void runProcessLoop() { try { @@ -633,6 +655,14 @@ public class Scheduler implements Runnable { argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); } + private PeriodicShardSyncManager buildPeriodicShardSyncManager() { + final ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, + cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, + metricsFactory); + return new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), + leaderDecider, shardSyncTask, metricsFactory); + } + /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. *

diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 8be1bb8f..3bf68829 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 @@ -89,6 +89,8 @@ public class SchedulerTest { private final String applicationName = "applicationName"; private final String streamName = "streamName"; private final String namespace = "testNamespace"; + private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 5 * 1000L; + private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; private Scheduler scheduler; private ShardRecordProcessorFactory shardRecordProcessorFactory; @@ -265,6 +267,43 @@ public class SchedulerTest { verify(shardDetector, times(maxInitializationAttempts)).listShards(); } + @Test + public final void testInitializationWaitsWhenLeaseTableIsEmpty() throws Exception { + final int maxInitializationAttempts = 1; + coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts); + coordinatorConfig.skipShardSyncAtWorkerInitializationIfLeasesExist(false); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + + doNothing().when(leaseCoordinator).initialize(); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + + long startTime = System.currentTimeMillis(); + scheduler.waitUntilLeaseTableIsReady(); + long endTime = System.currentTimeMillis(); + + assertTrue(endTime - startTime > MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + assertTrue(endTime - startTime < MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + } + + @Test + public final void testInitializationDoesntWaitWhenLeaseTableIsNotEmpty() throws Exception { + final int maxInitializationAttempts = 1; + coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts); + coordinatorConfig.skipShardSyncAtWorkerInitializationIfLeasesExist(false); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + + doNothing().when(leaseCoordinator).initialize(); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); + + long startTime = System.currentTimeMillis(); + scheduler.waitUntilLeaseTableIsReady(); + long endTime = System.currentTimeMillis(); + + assertTrue(endTime - startTime < MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + } + @Test public final void testSchedulerShutdown() { scheduler.shutdown(); From f625aea63de4755ccda70f449213a276ec9d57aa Mon Sep 17 00:00:00 2001 From: Jegosh John Date: Fri, 6 Mar 2020 11:37:29 -0800 Subject: [PATCH 004/159] Fixing scheduler initialization waitWhenLeaseTableEmpty unit test --- .../software/amazon/kinesis/coordinator/SchedulerTest.java | 3 ++- 1 file changed, 2 insertions(+), 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 3bf68829..3a0ad054 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 @@ -91,6 +91,7 @@ public class SchedulerTest { private final String namespace = "testNamespace"; private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 5 * 1000L; private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; + private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; private Scheduler scheduler; private ShardRecordProcessorFactory shardRecordProcessorFactory; @@ -283,7 +284,7 @@ public class SchedulerTest { long endTime = System.currentTimeMillis(); assertTrue(endTime - startTime > MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); - assertTrue(endTime - startTime < MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + assertTrue(endTime - startTime < (MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS + LEASE_TABLE_CHECK_FREQUENCY_MILLIS)); } @Test From 38c64b8624e839a266c7dab2b1425f3cd82bd285 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 11 Feb 2020 13:05:26 -0800 Subject: [PATCH 005/159] Fixing doc comment shard structure format for HierarchicalShardSyncerTest --- .../leases/HierarchicalShardSyncerTest.java | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index 23d2e423..4dc3fdd3 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 @@ -844,11 +844,11 @@ public class HierarchicalShardSyncerTest { // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) // * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) +// * 0 1 2 3 4 5- shards till epoch 102 +// * \ / \ / | | +// * 6 7 4 5- shards from epoch 103 - 205 +// * \ / | /\ +// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) // * Current leases: (3, 4, 5) // */ @Test @@ -879,11 +879,11 @@ public class HierarchicalShardSyncerTest { // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) // * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) +// * 0 1 2 3 4 5- shards till epoch 102 +// * \ / \ / | | +// * 6 7 4 5- shards from epoch 103 - 205 +// * \ / | /\ +// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) // * Current leases: (4, 5, 7) // */ @Test @@ -912,11 +912,11 @@ public class HierarchicalShardSyncerTest { // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) // * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) +// * 0 1 2 3 4 5- shards till epoch 102 +// * \ / \ / | | +// * 6 7 4 5- shards from epoch 103 - 205 +// * \ / | /\ +// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) // * Current leases: (3, 4, 5) // */ @Test @@ -947,11 +947,11 @@ public class HierarchicalShardSyncerTest { // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) // * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) +// * 0 1 2 3 4 5- shards till epoch 102 +// * \ / \ / | | +// * 6 7 4 5- shards from epoch 103 - 205 +// * \ / | /\ +// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) // * Current leases: (4, 5, 7) // */ @Test @@ -1148,11 +1148,11 @@ public class HierarchicalShardSyncerTest { // /* // * Helper method to construct a shard list for graph B. Graph B is defined below. // * Shard structure (x-axis is epochs): -// * 0 3 6 9 +// * 0 3 6 9 // * \ / \ / \ / -// * 2 5 8 +// * 2 5 8 // * / \ / \ / \ -// * 1 4 7 10 +// * 1 4 7 10 // */ private List constructShardListForGraphB() { final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("1000", "1049"); From d6e2e0b324fb9ff48bd743134b45a3978a7ecde5 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Sat, 15 Feb 2020 00:29:38 -0800 Subject: [PATCH 006/159] Multistream tracker v1 --- .../dynamodb/DynamoDBCheckpointer.java | 46 +++---- .../amazon/kinesis/common/ConfigsBuilder.java | 9 +- .../amazon/kinesis/common/StreamConfig.java | 14 ++ .../amazon/kinesis/coordinator/Scheduler.java | 71 ++++++---- .../kinesis/leases/CompositeLeaseKey.java | 46 +++++++ .../kinesis/leases/LeaseManagementConfig.java | 70 +++++++++- .../leases/LeaseManagementFactory.java | 10 ++ .../kinesis/leases/LeaseSerializer.java | 6 +- .../kinesis/leases/MultiStreamLease.java | 91 +++++++++++++ .../amazon/kinesis/leases/ShardInfo.java | 17 ++- .../dynamodb/DynamoDBLeaseCoordinator.java | 19 ++- .../DynamoDBLeaseManagementFactory.java | 121 ++++++++++++++++-- .../dynamodb/DynamoDBLeaseSerializer.java | 26 ++-- .../DynamoDBMultiStreamLeaseSerializer.java | 47 +++++++ .../kinesis/processor/Checkpointer.java | 16 +-- .../kinesis/processor/MultiStreamTracker.java | 27 ++++ .../ShardRecordProcessorFactory.java | 12 +- .../kinesis/retrieval/RetrievalConfig.java | 6 + .../checkpoint/InMemoryCheckpointer.java | 28 ++-- 19 files changed, 577 insertions(+), 105 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java index b0ce7675..fb7a6fc7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java @@ -54,54 +54,54 @@ public class DynamoDBCheckpointer implements Checkpointer { private String operation; @Override - public void setCheckpoint(final String shardId, final ExtendedSequenceNumber checkpointValue, + public void setCheckpoint(final String leaseKey, final ExtendedSequenceNumber checkpointValue, final String concurrencyToken) throws KinesisClientLibException { try { - boolean wasSuccessful = setCheckpoint(shardId, checkpointValue, UUID.fromString(concurrencyToken)); + boolean wasSuccessful = setCheckpoint(leaseKey, checkpointValue, UUID.fromString(concurrencyToken)); if (!wasSuccessful) { throw new ShutdownException("Can't update checkpoint - instance doesn't hold the lease for this shard"); } } catch (ProvisionedThroughputException e) { throw new ThrottlingException("Got throttled while updating checkpoint.", e); } catch (InvalidStateException e) { - String message = "Unable to save checkpoint for shardId " + shardId; + String message = "Unable to save checkpoint for shardId " + leaseKey; log.error(message, e); throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e); } catch (DependencyException e) { - throw new KinesisClientLibDependencyException("Unable to save checkpoint for shardId " + shardId, e); + throw new KinesisClientLibDependencyException("Unable to save checkpoint for shardId " + leaseKey, e); } } @Override - public ExtendedSequenceNumber getCheckpoint(final String shardId) throws KinesisClientLibException { + public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws KinesisClientLibException { try { - return leaseRefresher.getLease(shardId).checkpoint(); + return leaseRefresher.getLease(leaseKey).checkpoint(); } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { - String message = "Unable to fetch checkpoint for shardId " + shardId; + String message = "Unable to fetch checkpoint for shardId " + leaseKey; log.error(message, e); throw new KinesisClientLibIOException(message, e); } } @Override - public Checkpoint getCheckpointObject(final String shardId) throws KinesisClientLibException { + public Checkpoint getCheckpointObject(final String leaseKey) throws KinesisClientLibException { try { - Lease lease = leaseRefresher.getLease(shardId); - log.debug("[{}] Retrieved lease => {}", shardId, lease); + Lease lease = leaseRefresher.getLease(leaseKey); + log.debug("[{}] Retrieved lease => {}", leaseKey, lease); return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint()); } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { - String message = "Unable to fetch checkpoint for shardId " + shardId; + String message = "Unable to fetch checkpoint for shardId " + leaseKey; log.error(message, e); throw new KinesisClientLibIOException(message, e); } } @Override - public void prepareCheckpoint(final String shardId, final ExtendedSequenceNumber pendingCheckpoint, + public void prepareCheckpoint(final String leaseKey, final ExtendedSequenceNumber pendingCheckpoint, final String concurrencyToken) throws KinesisClientLibException { try { boolean wasSuccessful = - prepareCheckpoint(shardId, pendingCheckpoint, UUID.fromString(concurrencyToken)); + prepareCheckpoint(leaseKey, pendingCheckpoint, UUID.fromString(concurrencyToken)); if (!wasSuccessful) { throw new ShutdownException( "Can't prepare checkpoint - instance doesn't hold the lease for this shard"); @@ -109,21 +109,21 @@ public class DynamoDBCheckpointer implements Checkpointer { } catch (ProvisionedThroughputException e) { throw new ThrottlingException("Got throttled while preparing checkpoint.", e); } catch (InvalidStateException e) { - String message = "Unable to prepare checkpoint for shardId " + shardId; + String message = "Unable to prepare checkpoint for shardId " + leaseKey; log.error(message, e); throw new software.amazon.kinesis.exceptions.InvalidStateException(message, e); } catch (DependencyException e) { - throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + shardId, e); + throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + leaseKey, e); } } @VisibleForTesting - public boolean setCheckpoint(String shardId, ExtendedSequenceNumber checkpoint, UUID concurrencyToken) + public boolean setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpoint, UUID concurrencyToken) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - Lease lease = leaseCoordinator.getCurrentlyHeldLease(shardId); + Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey); if (lease == null) { log.info("Worker {} could not update checkpoint for shard {} because it does not hold the lease", - leaseCoordinator.workerIdentifier(), shardId); + leaseCoordinator.workerIdentifier(), leaseKey); return false; } @@ -131,20 +131,20 @@ public class DynamoDBCheckpointer implements Checkpointer { lease.pendingCheckpoint(null); lease.ownerSwitchesSinceCheckpoint(0L); - return leaseCoordinator.updateLease(lease, concurrencyToken, operation, shardId); + return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } - boolean prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken) + boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - Lease lease = leaseCoordinator.getCurrentlyHeldLease(shardId); + Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey); if (lease == null) { log.info("Worker {} could not prepare checkpoint for shard {} because it does not hold the lease", - leaseCoordinator.workerIdentifier(), shardId); + leaseCoordinator.workerIdentifier(), leaseKey); return false; } lease.pendingCheckpoint(Objects.requireNonNull(pendingCheckpoint, "pendingCheckpoint should not be null")); - return leaseCoordinator.updateLease(lease, concurrencyToken, operation, shardId); + return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java index c7f56d8d..7026e34c 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 @@ -30,6 +30,7 @@ import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.RetrievalConfig; /** @@ -108,6 +109,8 @@ public class ConfigsBuilder { return namespace; } + private MultiStreamTracker multiStreamTracker; + /** * Creates a new instance of CheckpointConfig * @@ -170,6 +173,10 @@ public class ConfigsBuilder { * @return RetrievalConfig */ public RetrievalConfig retrievalConfig() { - return new RetrievalConfig(kinesisClient(), streamName(), applicationName()); + final RetrievalConfig retrievalConfig = new RetrievalConfig(kinesisClient(), streamName(), applicationName()); + if(this.multiStreamTracker != null) { + retrievalConfig.multiStreamTracker(multiStreamTracker); + } + return retrievalConfig; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java new file mode 100644 index 00000000..55451709 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java @@ -0,0 +1,14 @@ +package software.amazon.kinesis.common; + +import lombok.AccessLevel; +import lombok.Data; +import lombok.experimental.Accessors; +import lombok.experimental.FieldDefaults; + +@Data +@Accessors(fluent = true) +@FieldDefaults(makeFinal=true, level= AccessLevel.PRIVATE) +public class StreamConfig { + String streamName; + InitialPositionInStreamExtended initialPositionInStreamExtended; +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index eaeb5a1c..b6b7fab3 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 @@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import io.reactivex.plugins.RxJavaPlugins; import lombok.AccessLevel; import lombok.Getter; @@ -36,6 +37,8 @@ import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -43,6 +46,7 @@ import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.LeaseSerializer; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardPrioritization; @@ -50,6 +54,8 @@ import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; +import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer; +import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer; import software.amazon.kinesis.leases.exceptions.LeasingException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; @@ -66,6 +72,7 @@ import software.amazon.kinesis.processor.Checkpointer; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShutdownNotificationAware; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; @@ -100,7 +107,7 @@ public class Scheduler implements Runnable { private final DiagnosticEventHandler diagnosticEventHandler; // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; - private final ShardSyncTaskManager shardSyncTaskManager; +// private final ShardSyncTaskManager shardSyncTaskManager; private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; @@ -110,11 +117,12 @@ public class Scheduler implements Runnable { private final MetricsFactory metricsFactory; private final long failoverTimeMillis; private final long taskBackoffTimeMillis; - private final String streamName; + private final List listOfStreams; + private final MultiStreamTracker multiStreamTracker; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; - private final ShardDetector shardDetector; +// private final ShardDetector shardDetector; private final boolean ignoreUnexpetedChildShards; private final AggregatorUtil aggregatorUtil; private final HierarchicalShardSyncer hierarchicalShardSyncer; @@ -170,9 +178,18 @@ public class Scheduler implements Runnable { this.retrievalConfig = retrievalConfig; this.applicationName = this.coordinatorConfig.applicationName(); + this.multiStreamTracker = this.retrievalConfig.multiStreamTracker(); + this.listOfStreams = this.multiStreamTracker == null ? + ImmutableList.of(this.retrievalConfig.streamName()) : + this.multiStreamTracker.listStreamsToProcess(); + Validate.isTrue(!CollectionUtils.isNullOrEmpty(this.listOfStreams), "No stream configured to process."); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); - this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory() + // Determine leaseSerializer based on MultiStreamTracker + final LeaseSerializer leaseSerializer = this.multiStreamTracker == null ? + new DynamoDBMultiStreamLeaseSerializer() : + new DynamoDBLeaseSerializer(); + this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer) .createLeaseCoordinator(this.metricsFactory); this.leaseRefresher = this.leaseCoordinator.leaseRefresher(); @@ -191,8 +208,8 @@ public class Scheduler implements Runnable { this.diagnosticEventFactory = diagnosticEventFactory; this.diagnosticEventHandler = new DiagnosticEventLogger(); - this.shardSyncTaskManager = this.leaseManagementConfig.leaseManagementFactory() - .createShardSyncTaskManager(this.metricsFactory); +// this.shardSyncTaskManager = this.leaseManagementConfig.leaseManagementFactory() +// .createShardSyncTaskManager(this.metricsFactory); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -214,10 +231,9 @@ public class Scheduler implements Runnable { this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis(); // this.retryGetRecordsInSeconds = this.retrievalConfig.retryGetRecordsInSeconds(); // this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool(); - this.streamName = this.retrievalConfig.streamName(); this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis(); this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts(); - this.shardDetector = this.shardSyncTaskManager.shardDetector(); +// this.shardDetector = this.shardSyncTaskManager.shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); @@ -264,28 +280,35 @@ public class Scheduler implements Runnable { log.info("Initializing LeaseCoordinator"); leaseCoordinator.initialize(); - TaskResult result = null; + TaskResult result; if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { - log.info("Syncing Kinesis shard info"); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, - metricsFactory); - result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); + // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing + // TODO: for already synced streams + for(String streamName : this.listOfStreams) { + log.info("Syncing Kinesis shard info"); + ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, + initialPosition, cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, + hierarchicalShardSyncer, metricsFactory); + result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); + // Throwing the exception, to prevent further syncs for other stream. + if (result.getException() != null) { + log.error("Caught exception when sync'ing info for " + streamName, result.getException()); + throw result.getException(); + } + } } else { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); } - if (result == null || result.getException() == null) { - if (!leaseCoordinator.isRunning()) { - log.info("Starting LeaseCoordinator"); - leaseCoordinator.start(); - } else { - log.info("LeaseCoordinator is already running. No need to start it."); - } - isDone = true; + // If we reach this point, then we either skipped the lease sync or did not have any exception + // for any of the shard sync in the previous attempt. + if (!leaseCoordinator.isRunning()) { + log.info("Starting LeaseCoordinator"); + leaseCoordinator.start(); } else { - lastException = result.getException(); + log.info("LeaseCoordinator is already running. No need to start it."); } + isDone = true; } catch (LeasingException e) { log.error("Caught exception when initializing LeaseCoordinator", e); lastException = e; @@ -591,7 +614,7 @@ public class Scheduler implements Runnable { ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, checkpoint); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, - streamName, + shardInfo.streamName(), leaseCoordinator, executorService, cache, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java new file mode 100644 index 00000000..7092ceaa --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java @@ -0,0 +1,46 @@ +package software.amazon.kinesis.leases; + +import lombok.Getter; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.Validate; + +import java.util.Optional; + +public class CompositeLeaseKey { + +// private static final String LEASE_TOKEN_SEPERATOR = ":"; +// +// private String streamName; +// +// @Getter +// private String shardId; +// +// public CompositeLeaseKey(String shardId) { +// this(null, shardId); +// } +// +// public CompositeLeaseKey(String streamName, String shardId) { +// this.streamName = streamName; +// this.shardId = shardId; +// } +// +// public Optional getStreamName() { +// return Optional.ofNullable(streamName); +// } +// +// public String getLeaseKey(boolean isMultiStreamingEnabled) { +// Validate.isTrue(!(isMultiStreamingEnabled && StringUtils.isEmpty(streamName)), +// "Empty stream name found while multiStreaming is enabled"); +// return isMultiStreamingEnabled ? StringUtils.joinWith(LEASE_TOKEN_SEPERATOR, streamName, shardId) : shardId; +// } +// +// public static CompositeLeaseKey getLeaseKey(String leaseKey) { +// Validate.notNull(leaseKey); +// String leaseTokens[] = leaseKey.split(LEASE_TOKEN_SEPERATOR); +// Validate.inclusiveBetween(1, 2, leaseTokens.length); +// return leaseTokens.length == 2 ? +// new CompositeLeaseKey(leaseTokens[0], leaseTokens[1]) : +// new CompositeLeaseKey(leaseTokens[0]); +// } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java index 20e0aa8f..3c71934b 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 @@ -16,6 +16,8 @@ package software.amazon.kinesis.leases; import java.time.Duration; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; @@ -27,6 +29,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; +import org.apache.commons.lang3.Validate; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -36,6 +39,7 @@ import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseManagementFactory; import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.NullMetricsFactory; +import software.amazon.kinesis.processor.MultiStreamTracker; /** * Used by the KCL to configure lease management. @@ -71,7 +75,7 @@ public class LeaseManagementConfig { * Name of the Kinesis Data Stream to read records from. */ @NonNull - private final String streamName; + private String streamName; /** * Used to distinguish different workers/processes of a KCL application. * @@ -116,7 +120,7 @@ public class LeaseManagementConfig { * *

Default value: {@link Integer#MAX_VALUE}

*/ - private int maxLeasesForWorker = Integer.MAX_VALUE;; + private int maxLeasesForWorker = Integer.MAX_VALUE; /** * Max leases to steal from another worker at one time (for load balancing). @@ -182,6 +186,24 @@ public class LeaseManagementConfig { private MetricsFactory metricsFactory = new NullMetricsFactory(); + @Deprecated + public LeaseManagementConfig(String tableName, DynamoDbAsyncClient dynamoDBClient, KinesisAsyncClient kinesisClient, + String streamName, String workerIdentifier) { + this.tableName = tableName; + this.dynamoDBClient = dynamoDBClient; + this.kinesisClient = kinesisClient; + this.streamName = streamName; + this.workerIdentifier = workerIdentifier; + } + + public LeaseManagementConfig(String tableName, DynamoDbAsyncClient dynamoDBClient, KinesisAsyncClient kinesisClient, + String workerIdentifier) { + this.tableName = tableName; + this.dynamoDBClient = dynamoDBClient; + this.kinesisClient = kinesisClient; + this.workerIdentifier = workerIdentifier; + } + /** * Returns the metrics factory. * @@ -244,9 +266,10 @@ public class LeaseManagementConfig { private LeaseManagementFactory leaseManagementFactory; + @Deprecated public LeaseManagementFactory leaseManagementFactory() { - if (leaseManagementFactory == null) { - leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), + Validate.notEmpty(streamName(), "Stream name is empty"); + return new DynamoDBLeaseManagementFactory(kinesisClient(), streamName(), dynamoDBClient(), tableName(), @@ -271,8 +294,43 @@ public class LeaseManagementConfig { initialLeaseTableWriteCapacity(), hierarchicalShardSyncer(), tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode()); - } - return leaseManagementFactory; } + public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer) { + return new DynamoDBLeaseManagementFactory(kinesisClient(), + dynamoDBClient(), + tableName(), + workerIdentifier(), + executorService(), + failoverTimeMillis(), + epsilonMillis(), + maxLeasesForWorker(), + maxLeasesToStealAtOneTime(), + maxLeaseRenewalThreads(), + cleanupLeasesUponShardCompletion(), + ignoreUnexpectedChildShards(), + shardSyncIntervalMillis(), + consistentReads(), + listShardsBackoffTimeInMillis(), + maxListShardsRetryAttempts(), + maxCacheMissesBeforeReload(), + listShardsCacheAllowedAgeInSeconds(), + cacheMissWarningModulus(), + initialLeaseTableReadCapacity(), + initialLeaseTableWriteCapacity(), + hierarchicalShardSyncer(), + tableCreatorCallback(), + dynamoDbRequestTimeout(), + billingMode(), + leaseSerializer); + } + +// private InitialPositionInStreamExtended getInitialPositionExtendedForStream(String streamName) { +// return multiStreamTracker() == null ? +// initialPositionInStream() : +// multiStreamTracker().initialPositionInStreamExtended(streamName) == null ? +// initialPositionInStream() : +// multiStreamTracker().initialPositionInStreamExtended(streamName); +// } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java index 72f48fea..37f66258 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java @@ -15,6 +15,7 @@ package software.amazon.kinesis.leases; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.metrics.MetricsFactory; @@ -26,7 +27,16 @@ public interface LeaseManagementFactory { ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory); + default ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { + throw new UnsupportedOperationException(); + } + DynamoDBLeaseRefresher createLeaseRefresher(); ShardDetector createShardDetector(); + + default ShardDetector createShardDetector(StreamConfig streamConfig) { + throw new UnsupportedOperationException(); + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java index b8aa0339..95b98399 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java @@ -23,7 +23,6 @@ import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue; import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; -import software.amazon.kinesis.leases.Lease; /** * Utility class that manages the mapping of Lease objects/operations to records in DynamoDB. @@ -46,6 +45,11 @@ public interface LeaseSerializer { */ Lease fromDynamoRecord(Map dynamoRecord); + + default Lease fromDynamoRecord(Map dynamoRecord, Lease leaseToUpdate) { + throw new UnsupportedOperationException(); + } + /** * @param lease * @return the attribute value map representing a Lease's hash key given a Lease object. diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java new file mode 100644 index 00000000..9878e32c --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/MultiStreamLease.java @@ -0,0 +1,91 @@ +package software.amazon.kinesis.leases; + +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.NonNull; +import lombok.Setter; +import lombok.experimental.Accessors; +import org.apache.commons.lang3.Validate; + +import java.util.Objects; + +import static com.google.common.base.Verify.verifyNotNull; + +@Setter +@NoArgsConstructor +@Getter +@Accessors(fluent = true) +public class MultiStreamLease extends Lease { + + @NonNull private String streamName; + @NonNull private String shardId; + + public MultiStreamLease(Lease other) { + super(other); + MultiStreamLease casted = validateAndCast(other); + streamName(casted.streamName); + shardId(casted.shardId); + } + + @Override + public void update(Lease other) { + MultiStreamLease casted = validateAndCast(other); + super.update(casted); + streamName(casted.streamName); + shardId(casted.shardId); + } + + public static String getLeaseKey(String streamName, String shardId) { + verifyNotNull(streamName, "streamName should not be null"); + verifyNotNull(shardId, "shardId should not be null"); + return streamName + ":" + shardId; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), streamName); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (!(obj instanceof MultiStreamLease)) { + return false; + } + MultiStreamLease other = (MultiStreamLease) obj; + if (streamName == null) { + if (other.streamName != null) { + return false; + } + } else if (!streamName.equals(other.streamName)) { + return false; + } + return true; + } + + /** + * Returns a deep copy of this object. Type-unsafe - there aren't good mechanisms for copy-constructing generics. + * + * @return A deep copy of this object. + */ + @Override + public MultiStreamLease copy() { + return new MultiStreamLease(this); + } + + /** + * Validate and cast the lease to MultiStream lease + * @param lease + * @return MultiStreamLease + */ + public static MultiStreamLease validateAndCast(Lease lease) { + Validate.isInstanceOf(MultiStreamLease.class, lease); + return (MultiStreamLease) lease; + } + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 89b8f94a..75070d8c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -36,6 +36,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @ToString public class ShardInfo { + private final String streamName; private final String shardId; private final String concurrencyToken; // Sorted list of parent shardIds. @@ -54,11 +55,18 @@ public class ShardInfo { * @param checkpoint * the latest checkpoint from lease */ - // TODO: check what values can be null public ShardInfo(@NonNull final String shardId, final String concurrencyToken, final Collection parentShardIds, final ExtendedSequenceNumber checkpoint) { + this(shardId, concurrencyToken, parentShardIds, checkpoint, null); + } + + public ShardInfo(@NonNull final String shardId, + final String concurrencyToken, + final Collection parentShardIds, + final ExtendedSequenceNumber checkpoint, + final String streamName) { this.shardId = shardId; this.concurrencyToken = concurrencyToken; this.parentShardIds = new LinkedList<>(); @@ -69,6 +77,7 @@ public class ShardInfo { // This makes it easy to check for equality in ShardInfo.equals method. Collections.sort(this.parentShardIds); this.checkpoint = checkpoint; + this.streamName = streamName; } /** @@ -94,7 +103,8 @@ public class ShardInfo { */ @Override public int hashCode() { - return new HashCodeBuilder().append(concurrencyToken).append(parentShardIds).append(shardId).toHashCode(); + return new HashCodeBuilder() + .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamName).toHashCode(); } /** @@ -118,7 +128,8 @@ public class ShardInfo { } ShardInfo other = (ShardInfo) obj; return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken) - .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId).isEquals(); + .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId) + .append(streamName, other.streamName).isEquals(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java index fe31d996..01cd6683 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 @@ -33,11 +33,13 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import lombok.extern.slf4j.Slf4j; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.leases.CompositeLeaseKey; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseRenewer; import software.amazon.kinesis.leases.LeaseTaker; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; @@ -377,9 +379,22 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { return leases.stream().map(DynamoDBLeaseCoordinator::convertLeaseToAssignment).collect(Collectors.toList()); } + // TODO : Halo : Reenable for backward compatibility +// public static ShardInfo convertLeaseToAssignment(final Lease lease) { +// return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), +// lease.checkpoint()); +// } + + // TODO : Support Shard public static ShardInfo convertLeaseToAssignment(final Lease lease) { - return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), - lease.checkpoint()); + if (lease instanceof MultiStreamLease) { + return new ShardInfo(((MultiStreamLease) lease).shardId(), lease.concurrencyToken().toString(), lease.parentShardIds(), + lease.checkpoint(), ((MultiStreamLease) lease).streamName()); + } else { + return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), + lease.checkpoint()); + } + } /** diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java index c2ade429..124d28ca 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,6 +16,9 @@ package software.amazon.kinesis.leases.dynamodb; import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.ExecutorService; import lombok.Data; @@ -23,13 +26,17 @@ import lombok.NonNull; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.KinesisShardDetector; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseManagementFactory; +import software.amazon.kinesis.leases.LeaseSerializer; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.metrics.MetricsFactory; @@ -44,8 +51,6 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @NonNull private final KinesisAsyncClient kinesisClient; @NonNull - private final String streamName; - @NonNull private final DynamoDbAsyncClient dynamoDBClient; @NonNull private final String tableName; @@ -54,9 +59,11 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @NonNull private final ExecutorService executorService; @NonNull - private final InitialPositionInStreamExtended initialPositionInStream; - @NonNull private final HierarchicalShardSyncer hierarchicalShardSyncer; + @NonNull + private final LeaseSerializer leaseSerializer; + @NonNull + private StreamConfig streamConfig; private final long failoverTimeMillis; private final long epsilonMillis; @@ -309,6 +316,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param dynamoDbRequestTimeout * @param billingMode */ + @Deprecated public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final String streamName, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final InitialPositionInStreamExtended initialPositionInStream, @@ -321,13 +329,83 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode) { + + this(kinesisClient, new StreamConfig(streamName, initialPositionInStream), dynamoDBClient, tableName, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, + cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); + } + + /** + * Constructor. + * + * @param kinesisClient + * @param streamConfig + * @param dynamoDBClient + * @param tableName + * @param workerIdentifier + * @param executorService + * @param failoverTimeMillis + * @param epsilonMillis + * @param maxLeasesForWorker + * @param maxLeasesToStealAtOneTime + * @param maxLeaseRenewalThreads + * @param cleanupLeasesUponShardCompletion + * @param ignoreUnexpectedChildShards + * @param shardSyncIntervalMillis + * @param consistentReads + * @param listShardsBackoffTimeMillis + * @param maxListShardsRetryAttempts + * @param maxCacheMissesBeforeReload + * @param listShardsCacheAllowedAgeInSeconds + * @param cacheMissWarningModulus + * @param initialLeaseTableReadCapacity + * @param initialLeaseTableWriteCapacity + * @param hierarchicalShardSyncer + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + */ + public 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 hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { + this(kinesisClient, dynamoDBClient, tableName, + workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, + maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, + cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer); + this.streamConfig = streamConfig; + } + + public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, + final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, + final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, + final int maxLeasesForWorker, final int maxLeasesToStealAtOneTime, final int maxLeaseRenewalThreads, + final boolean cleanupLeasesUponShardCompletion, final boolean ignoreUnexpectedChildShards, + final long shardSyncIntervalMillis, final boolean consistentReads, final long listShardsBackoffTimeMillis, + final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, + final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, + final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, + final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { this.kinesisClient = kinesisClient; - this.streamName = streamName; this.dynamoDBClient = dynamoDBClient; this.tableName = tableName; this.workerIdentifier = workerIdentifier; this.executorService = executorService; - this.initialPositionInStream = initialPositionInStream; this.failoverTimeMillis = failoverTimeMillis; this.epsilonMillis = epsilonMillis; this.maxLeasesForWorker = maxLeasesForWorker; @@ -348,6 +426,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.tableCreatorCallback = tableCreatorCallback; this.dynamoDbRequestTimeout = dynamoDbRequestTimeout; this.billingMode = billingMode; + this.leaseSerializer = leaseSerializer; } @Override @@ -364,11 +443,24 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { metricsFactory); } - @Override + @Override @Deprecated public ShardSyncTaskManager createShardSyncTaskManager(@NonNull final MetricsFactory metricsFactory) { return new ShardSyncTaskManager(this.createShardDetector(), this.createLeaseRefresher(), - initialPositionInStream, + streamConfig.initialPositionInStreamExtended(), + cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, + shardSyncIntervalMillis, + executorService, + hierarchicalShardSyncer, + metricsFactory); + } + + @Override + public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { + return new ShardSyncTaskManager(this.createShardDetector(streamConfig), + this.createLeaseRefresher(), + streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, @@ -379,13 +471,20 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @Override public DynamoDBLeaseRefresher createLeaseRefresher() { - return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, new DynamoDBLeaseSerializer(), consistentReads, + return new DynamoDBLeaseRefresher(tableName, dynamoDBClient, leaseSerializer, consistentReads, tableCreatorCallback, dynamoDbRequestTimeout, billingMode); } - @Override + @Override @Deprecated public ShardDetector createShardDetector() { - return new KinesisShardDetector(kinesisClient, streamName, listShardsBackoffTimeMillis, + return new KinesisShardDetector(kinesisClient, streamConfig.streamName(), + listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, + maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout); + } + + @Override + public ShardDetector createShardDetector(StreamConfig streamConfig) { + return new KinesisShardDetector(kinesisClient, streamConfig.streamName(), listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java index b97738ca..a02e2a6e 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 @@ -80,28 +80,32 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { @Override public Lease fromDynamoRecord(final Map dynamoRecord) { - Lease result = new Lease(); - result.leaseKey(DynamoUtils.safeGetString(dynamoRecord, LEASE_KEY_KEY)); - result.leaseOwner(DynamoUtils.safeGetString(dynamoRecord, LEASE_OWNER_KEY)); - result.leaseCounter(DynamoUtils.safeGetLong(dynamoRecord, LEASE_COUNTER_KEY)); + final Lease result = new Lease(); + return fromDynamoRecord(dynamoRecord, result); + } - result.ownerSwitchesSinceCheckpoint(DynamoUtils.safeGetLong(dynamoRecord, OWNER_SWITCHES_KEY)); - result.checkpoint( + @Override + public Lease fromDynamoRecord(Map dynamoRecord, Lease leaseToUpdate) { + leaseToUpdate.leaseKey(DynamoUtils.safeGetString(dynamoRecord, LEASE_KEY_KEY)); + leaseToUpdate.leaseOwner(DynamoUtils.safeGetString(dynamoRecord, LEASE_OWNER_KEY)); + leaseToUpdate.leaseCounter(DynamoUtils.safeGetLong(dynamoRecord, LEASE_COUNTER_KEY)); + + leaseToUpdate.ownerSwitchesSinceCheckpoint(DynamoUtils.safeGetLong(dynamoRecord, OWNER_SWITCHES_KEY)); + leaseToUpdate.checkpoint( new ExtendedSequenceNumber( DynamoUtils.safeGetString(dynamoRecord, CHECKPOINT_SEQUENCE_NUMBER_KEY), DynamoUtils.safeGetLong(dynamoRecord, CHECKPOINT_SUBSEQUENCE_NUMBER_KEY)) ); - result.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY)); + leaseToUpdate.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY)); if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) { - result.pendingCheckpoint( + leaseToUpdate.pendingCheckpoint( new ExtendedSequenceNumber( DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY), DynamoUtils.safeGetLong(dynamoRecord, PENDING_CHECKPOINT_SUBSEQUENCE_KEY)) ); } - - return result; + return leaseToUpdate; } @Override @@ -198,7 +202,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { return result; } - private AttributeValueUpdate putUpdate(AttributeValue attributeValue) { + protected AttributeValueUpdate putUpdate(AttributeValue attributeValue) { return AttributeValueUpdate.builder().value(attributeValue).action(AttributeAction.PUT).build(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java new file mode 100644 index 00000000..7525273f --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java @@ -0,0 +1,47 @@ +package software.amazon.kinesis.leases.dynamodb; + +import lombok.NoArgsConstructor; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; +import software.amazon.kinesis.leases.DynamoUtils; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.MultiStreamLease; + +import java.util.Map; + +import static software.amazon.kinesis.leases.MultiStreamLease.validateAndCast; + +@NoArgsConstructor +public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer { + + private static final String STREAM_NAME_KEY = "streamName"; + private static final String SHARD_ID_KEY = "shardId"; + + @Override + public Map toDynamoRecord(Lease lease) { + final MultiStreamLease multiStreamLease = validateAndCast(lease); + final Map result = super.toDynamoRecord(multiStreamLease); + result.put(STREAM_NAME_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamName())); + result.put(SHARD_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.shardId())); + return result; + } + + @Override + public MultiStreamLease fromDynamoRecord(Map dynamoRecord) { + final MultiStreamLease multiStreamLease = (MultiStreamLease) super + .fromDynamoRecord(dynamoRecord, new MultiStreamLease()); + multiStreamLease.streamName(DynamoUtils.safeGetString(dynamoRecord, STREAM_NAME_KEY)); + multiStreamLease.shardId(DynamoUtils.safeGetString(dynamoRecord, SHARD_ID_KEY)); + return multiStreamLease; + } + + + @Override + public Map getDynamoUpdateLeaseUpdate(Lease lease) { + final MultiStreamLease multiStreamLease = validateAndCast(lease); + final Map result = super.getDynamoUpdateLeaseUpdate(multiStreamLease); + result.put(STREAM_NAME_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamName()))); + result.put(SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.shardId()))); + return result; + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java index d3ecebc1..70cdd608 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java @@ -27,50 +27,50 @@ public interface Checkpointer { * Record a checkpoint for a shard (e.g. sequence and subsequence numbers of last record processed * by application). Upon failover, record processing is resumed from this point. * - * @param shardId Checkpoint is specified for this shard. + * @param leaseKey Checkpoint is specified for this shard. * @param checkpointValue Value of the checkpoint (e.g. Kinesis sequence number and subsequence number) * @param concurrencyToken Used with conditional writes to prevent stale updates * (e.g. if there was a fail over to a different record processor, we don't want to * overwrite it's checkpoint) * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint */ - void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken) + void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken) throws KinesisClientLibException; /** * Get the current checkpoint stored for the specified shard. Useful for checking that the parent shard * has been completely processed before we start processing the child shard. * - * @param shardId Current checkpoint for this shard is fetched + * @param leaseKey Current checkpoint for this shard is fetched * @return Current checkpoint for this shard, null if there is no record for this shard. * @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint */ - ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException; + ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException; /** * Get the current checkpoint stored for the specified shard, which holds the sequence numbers for the checkpoint * and pending checkpoint. Useful for checking that the parent shard has been completely processed before we start * processing the child shard. * - * @param shardId Current checkpoint for this shard is fetched + * @param leaseKey Current checkpoint for this shard is fetched * @return Current checkpoint object for this shard, null if there is no record for this shard. * @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint */ - Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException; + Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException; /** * Record intent to checkpoint for a shard. Upon failover, the pendingCheckpointValue will be passed to the new * ShardRecordProcessor's initialize() method. * - * @param shardId Checkpoint is specified for this shard. + * @param leaseKey Checkpoint is specified for this shard. * @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number) * @param concurrencyToken Used with conditional writes to prevent stale updates * (e.g. if there was a fail over to a different record processor, we don't want to * overwrite it's checkpoint) * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint */ - void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) + void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException; void operation(String operation); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java new file mode 100644 index 00000000..ac9e053b --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java @@ -0,0 +1,27 @@ +package software.amazon.kinesis.processor; + +import software.amazon.kinesis.common.InitialPositionInStreamExtended; + +import java.util.List; + +/** + * Interface for stream trackers. This is useful for KCL Workers that need + * to consume data from multiple streams. + */ +public interface MultiStreamTracker { + + /** + * Returns the list of streams that the Worker should consume data from. + * + * @return List of stream names + */ + List listStreamsToProcess(); + + /** + * Returns the initial position in stream to read from, for the given stream. + * @param streamName + * @return Initial position to read from, for the given stream + */ + InitialPositionInStreamExtended initialPositionInStreamExtended(String streamName); + +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java index b0559120..973b0393 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java @@ -24,5 +24,15 @@ public interface ShardRecordProcessorFactory { * * @return */ - ShardRecordProcessor shardRecordProcessor(); + // TODO : Halo : Reenable +// ShardRecordProcessor shardRecordProcessor(); + + /** + * Returns a new instance of the ShardRecordProcessor for a stream + * @param streamName + * @return ShardRecordProcessor + */ + default ShardRecordProcessor shardRecordProcessor(String streamName) { + throw new UnsupportedOperationException(); + } } 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 cfd19654..10b23641 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 @@ -21,6 +21,7 @@ import lombok.experimental.Accessors; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.fanout.FanOutConfig; /** @@ -51,6 +52,11 @@ public class RetrievalConfig { @NonNull private final String applicationName; + /** + * StreamTracker for multi streaming support + */ + private MultiStreamTracker multiStreamTracker; + /** * Backoff time between consecutive ListShards calls. * diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java index b8de6a1b..ebe933b9 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,14 +39,14 @@ public class InMemoryCheckpointer implements Checkpointer { * {@inheritDoc} */ @Override - public void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken) + public void setCheckpoint(String leaseKey, ExtendedSequenceNumber checkpointValue, String concurrencyToken) throws KinesisClientLibException { - checkpoints.put(shardId, checkpointValue); - flushpoints.put(shardId, checkpointValue); - pendingCheckpoints.remove(shardId); + checkpoints.put(leaseKey, checkpointValue); + flushpoints.put(leaseKey, checkpointValue); + pendingCheckpoints.remove(leaseKey); if (log.isDebugEnabled()) { - log.debug("shardId: {} checkpoint: {}", shardId, checkpointValue); + log.debug("shardId: {} checkpoint: {}", leaseKey, checkpointValue); } } @@ -55,25 +55,25 @@ public class InMemoryCheckpointer implements Checkpointer { * {@inheritDoc} */ @Override - public ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException { - ExtendedSequenceNumber checkpoint = flushpoints.get(shardId); - log.debug("checkpoint shardId: {} checkpoint: {}", shardId, checkpoint); + public ExtendedSequenceNumber getCheckpoint(String leaseKey) throws KinesisClientLibException { + ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); + log.debug("checkpoint shardId: {} checkpoint: {}", leaseKey, checkpoint); return checkpoint; } @Override - public void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException { - pendingCheckpoints.put(shardId, pendingCheckpoint); + pendingCheckpoints.put(leaseKey, pendingCheckpoint); } @Override - public Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException { - ExtendedSequenceNumber checkpoint = flushpoints.get(shardId); - ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(shardId); + public Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException { + ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); + ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(leaseKey); Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint); - log.debug("getCheckpointObject shardId: {}, {}", shardId, checkpointObj); + log.debug("getCheckpointObject shardId: {}, {}", leaseKey, checkpointObj); return checkpointObj; } From fd0e96a5d1af1a4fefc0d45b548ce50ddfc581de Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Sat, 15 Feb 2020 17:15:23 -0800 Subject: [PATCH 007/159] checkpointing --- .../amazon/kinesis/coordinator/Scheduler.java | 57 ++++++++++++------- .../amazon/kinesis/leases/ShardInfo.java | 5 +- .../dynamodb/DynamoDBLeaseCoordinator.java | 9 +-- .../kinesis/processor/MultiStreamTracker.java | 16 ++---- 4 files changed, 45 insertions(+), 42 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 b6b7fab3..fffc88a6 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 @@ -16,8 +16,11 @@ package software.amazon.kinesis.coordinator; import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -26,6 +29,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; @@ -39,9 +44,11 @@ import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.awssdk.utils.Either; 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.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -107,22 +114,21 @@ public class Scheduler implements Runnable { private final DiagnosticEventHandler diagnosticEventHandler; // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; -// private final ShardSyncTaskManager shardSyncTaskManager; + private final Function shardSyncTaskManagerProvider; + private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; private final GracefulShutdownCoordinator gracefulShutdownCoordinator; private final WorkerStateChangeListener workerStateChangeListener; - private final InitialPositionInStreamExtended initialPosition; private final MetricsFactory metricsFactory; private final long failoverTimeMillis; private final long taskBackoffTimeMillis; - private final List listOfStreams; - private final MultiStreamTracker multiStreamTracker; + private final Either applicationStreamTracker; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; -// private final ShardDetector shardDetector; + private final Function shardDetectorProvider; private final boolean ignoreUnexpetedChildShards; private final AggregatorUtil aggregatorUtil; private final HierarchicalShardSyncer hierarchicalShardSyncer; @@ -178,15 +184,17 @@ public class Scheduler implements Runnable { this.retrievalConfig = retrievalConfig; this.applicationName = this.coordinatorConfig.applicationName(); - this.multiStreamTracker = this.retrievalConfig.multiStreamTracker(); - this.listOfStreams = this.multiStreamTracker == null ? - ImmutableList.of(this.retrievalConfig.streamName()) : - this.multiStreamTracker.listStreamsToProcess(); - Validate.isTrue(!CollectionUtils.isNullOrEmpty(this.listOfStreams), "No stream configured to process."); + final MultiStreamTracker multiStreamTracker = this.retrievalConfig.multiStreamTracker(); + if(multiStreamTracker == null) { + this.applicationStreamTracker = Either.right(new StreamConfig(this.retrievalConfig.streamName(), + this.retrievalConfig.initialPositionInStreamExtended())); + } else { + this.applicationStreamTracker = Either.left(multiStreamTracker); + } this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); - // Determine leaseSerializer based on MultiStreamTracker - final LeaseSerializer leaseSerializer = this.multiStreamTracker == null ? + // Determine leaseSerializer based on availability of MultiStreamTracker. + final LeaseSerializer leaseSerializer = this.applicationStreamTracker.map(mst -> true, sc -> false) ? new DynamoDBMultiStreamLeaseSerializer() : new DynamoDBLeaseSerializer(); this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer) @@ -207,9 +215,11 @@ public class Scheduler implements Runnable { this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService(); this.diagnosticEventFactory = diagnosticEventFactory; this.diagnosticEventHandler = new DiagnosticEventLogger(); - -// this.shardSyncTaskManager = this.leaseManagementConfig.leaseManagementFactory() -// .createShardSyncTaskManager(this.metricsFactory); + // TODO : Halo : Handle case of no StreamConfig present in streamConfigMap() for the supplied streamName. + // TODO : Pass the immutable map here instead of using mst.streamConfigMap() + this.shardSyncTaskManagerProvider = streamName -> this.leaseManagementConfig + .leaseManagementFactory(leaseSerializer).createShardSyncTaskManager(this.metricsFactory, + applicationStreamTracker.map(mst -> mst.streamConfigMap().get(streamName), sc -> sc)); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -226,14 +236,13 @@ public class Scheduler implements Runnable { this.workerStateChangeListener = this.coordinatorConfig.coordinatorFactory() .createWorkerStateChangeListener(); } - this.initialPosition = retrievalConfig.initialPositionInStreamExtended(); 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.shardDetector = this.shardSyncTaskManager.shardDetector(); + this.shardDetectorProvider = streamName -> createOrGetShardSyncTaskManager(streamName).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); @@ -284,10 +293,14 @@ public class Scheduler implements Runnable { if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - for(String streamName : this.listOfStreams) { + final Map streamConfigMap = applicationStreamTracker + .map(mst -> mst.streamConfigMap(), sc -> Collections.singletonMap(sc.streamName(), sc)); + for(String streamName : streamConfigMap.keySet().stream().collect(Collectors.toList())) { log.info("Syncing Kinesis shard info"); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, - initialPosition, cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, + final StreamConfig streamConfig = streamConfigMap.get(streamName); + ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamName), + leaseRefresher, streamConfig.initialPositionInStreamExtended(), + cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, metricsFactory); result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); // Throwing the exception, to prevent further syncs for other stream. @@ -608,6 +621,10 @@ public class Scheduler implements Runnable { return consumer; } + private ShardSyncTaskManager createOrGetShardSyncTaskManager(String streamName) { + return streamToShardSyncTaskManagerMap.computeIfAbsent(streamName, s -> shardSyncTaskManagerProvider.apply(s)); + } + protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 75070d8c..6a6c73b7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -18,6 +18,7 @@ import java.util.Collection; import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.Optional; import org.apache.commons.lang3.builder.EqualsBuilder; import org.apache.commons.lang3.builder.HashCodeBuilder; @@ -36,7 +37,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @ToString public class ShardInfo { - private final String streamName; + private final Optional streamName; private final String shardId; private final String concurrencyToken; // Sorted list of parent shardIds. @@ -77,7 +78,7 @@ public class ShardInfo { // This makes it easy to check for equality in ShardInfo.equals method. Collections.sort(this.parentShardIds); this.checkpoint = checkpoint; - this.streamName = streamName; + this.streamName = Optional.ofNullable(streamName); } /** 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 01cd6683..2252da84 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 @@ -379,13 +379,7 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { return leases.stream().map(DynamoDBLeaseCoordinator::convertLeaseToAssignment).collect(Collectors.toList()); } - // TODO : Halo : Reenable for backward compatibility -// public static ShardInfo convertLeaseToAssignment(final Lease lease) { -// return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), -// lease.checkpoint()); -// } - - // TODO : Support Shard + // TODO : Halo : Check for better way public static ShardInfo convertLeaseToAssignment(final Lease lease) { if (lease instanceof MultiStreamLease) { return new ShardInfo(((MultiStreamLease) lease).shardId(), lease.concurrencyToken().toString(), lease.parentShardIds(), @@ -394,7 +388,6 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), lease.checkpoint()); } - } /** 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 ac9e053b..fae7e4cb 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 @@ -1,8 +1,8 @@ package software.amazon.kinesis.processor; -import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; -import java.util.List; +import java.util.Map; /** * Interface for stream trackers. This is useful for KCL Workers that need @@ -11,17 +11,9 @@ import java.util.List; public interface MultiStreamTracker { /** - * Returns the list of streams that the Worker should consume data from. + * Returns the map of streams and its associated stream specific config. * * @return List of stream names */ - List listStreamsToProcess(); - - /** - * Returns the initial position in stream to read from, for the given stream. - * @param streamName - * @return Initial position to read from, for the given stream - */ - InitialPositionInStreamExtended initialPositionInStreamExtended(String streamName); - + Map streamConfigMap(); } From 11c0ee75564d63e972d3356363b2bd7e3c98d75e Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 18 Feb 2020 22:14:39 -0800 Subject: [PATCH 008/159] Testcase fixes --- .../amazon/kinesis/coordinator/Scheduler.java | 62 +++++++++++------- .../kinesis/leases/LeaseManagementConfig.java | 65 +++++++++++-------- .../amazon/kinesis/leases/ShardInfo.java | 4 +- .../ShardRecordProcessorFactory.java | 5 +- .../kinesis/coordinator/SchedulerTest.java | 17 +++++ 5 files changed, 97 insertions(+), 56 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 fffc88a6..f564f47b 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 @@ -16,7 +16,6 @@ package software.amazon.kinesis.coordinator; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -34,7 +33,6 @@ import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; import io.reactivex.plugins.RxJavaPlugins; import lombok.AccessLevel; import lombok.Getter; @@ -42,12 +40,10 @@ import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.Validate; -import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.awssdk.utils.Either; +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.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; @@ -124,7 +120,8 @@ public class Scheduler implements Runnable { private final MetricsFactory metricsFactory; private final long failoverTimeMillis; private final long taskBackoffTimeMillis; - private final Either applicationStreamTracker; + private final Either appStreamTracker; + private final Map currentStreamConfigMap; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; @@ -186,15 +183,20 @@ public class Scheduler implements Runnable { this.applicationName = this.coordinatorConfig.applicationName(); final MultiStreamTracker multiStreamTracker = this.retrievalConfig.multiStreamTracker(); if(multiStreamTracker == null) { - this.applicationStreamTracker = Either.right(new StreamConfig(this.retrievalConfig.streamName(), - this.retrievalConfig.initialPositionInStreamExtended())); + final StreamConfig streamConfig = new StreamConfig(this.retrievalConfig.streamName(), + this.retrievalConfig.initialPositionInStreamExtended()); + this.appStreamTracker = Either.right(streamConfig); + this.currentStreamConfigMap = new HashMap() {{ + put(streamConfig.streamName(), streamConfig); + }}; } else { - this.applicationStreamTracker = Either.left(multiStreamTracker); + this.appStreamTracker = Either.left(multiStreamTracker); + this.currentStreamConfigMap = multiStreamTracker.streamConfigMap(); } this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); // Determine leaseSerializer based on availability of MultiStreamTracker. - final LeaseSerializer leaseSerializer = this.applicationStreamTracker.map(mst -> true, sc -> false) ? + final LeaseSerializer leaseSerializer = this.appStreamTracker.map(mst -> true, sc -> false) ? new DynamoDBMultiStreamLeaseSerializer() : new DynamoDBLeaseSerializer(); this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer) @@ -218,8 +220,8 @@ public class Scheduler implements Runnable { // TODO : Halo : Handle case of no StreamConfig present in streamConfigMap() for the supplied streamName. // TODO : Pass the immutable map here instead of using mst.streamConfigMap() this.shardSyncTaskManagerProvider = streamName -> this.leaseManagementConfig - .leaseManagementFactory(leaseSerializer).createShardSyncTaskManager(this.metricsFactory, - applicationStreamTracker.map(mst -> mst.streamConfigMap().get(streamName), sc -> sc)); + .leaseManagementFactory(leaseSerializer) + .createShardSyncTaskManager(this.metricsFactory, this.currentStreamConfigMap.get(streamName)); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -245,6 +247,7 @@ public class Scheduler implements Runnable { this.shardDetectorProvider = streamName -> createOrGetShardSyncTaskManager(streamName).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); + // TODO : Halo : Check if this needs to be per stream. this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); } @@ -293,11 +296,9 @@ public class Scheduler implements Runnable { if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - final Map streamConfigMap = applicationStreamTracker - .map(mst -> mst.streamConfigMap(), sc -> Collections.singletonMap(sc.streamName(), sc)); - for(String streamName : streamConfigMap.keySet().stream().collect(Collectors.toList())) { + for(String streamName : currentStreamConfigMap.keySet().stream().collect(Collectors.toList())) { log.info("Syncing Kinesis shard info"); - final StreamConfig streamConfig = streamConfigMap.get(streamName); + final StreamConfig streamConfig = currentStreamConfigMap.get(streamName); ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamName), leaseRefresher, streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, @@ -348,22 +349,27 @@ public class Scheduler implements Runnable { @VisibleForTesting void runProcessLoop() { try { - boolean foundCompletedShard = false; Set assignedShards = new HashSet<>(); + final Set completedShards = new HashSet<>(); for (ShardInfo shardInfo : getShardInfoForAssignments()) { ShardConsumer shardConsumer = createOrGetShardConsumer(shardInfo, processorConfig.shardRecordProcessorFactory()); if (shardConsumer.isShutdown() && shardConsumer.shutdownReason().equals(ShutdownReason.SHARD_END)) { - foundCompletedShard = true; + completedShards.add(shardInfo); } else { shardConsumer.executeLifecycle(); } assignedShards.add(shardInfo); } - if (foundCompletedShard) { - shardSyncTaskManager.syncShardAndLeaseInfo(); + for (ShardInfo completedShard : completedShards) { + final String streamName = completedShard.streamName() + .orElse(appStreamTracker.map(mst -> null, sc -> sc.streamName())); + Validate.notEmpty(streamName, "Stream name should not be null"); + if (createOrGetShardSyncTaskManager(streamName).syncShardAndLeaseInfo()) { + log.info("Found completed shard, initiated new ShardSyncTak for " + completedShard.toString()); + } } // clean up shard consumers for unassigned shards @@ -630,12 +636,20 @@ public class Scheduler implements Runnable { RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory); ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, checkpoint); + // The only case where streamName is not available will be when multistreamtracker not set. In this case, + // get the default stream name for the single stream application. + final String streamName = shardInfo.streamName().orElse(appStreamTracker.map(mst -> null, sc -> sc.streamName())); + Validate.notEmpty(streamName, "StreamName should not be empty"); + // Irrespective of single stream app or multi stream app, streamConfig should always be available. + // TODO: Halo : if not available, construct a default config ? + final StreamConfig streamConfig = currentStreamConfigMap.get(streamName); + Validate.notNull(streamConfig, "StreamConfig should not be empty"); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, - shardInfo.streamName(), + streamConfig.streamName(), leaseCoordinator, executorService, cache, - shardRecordProcessorFactory.shardRecordProcessor(), + shardRecordProcessorFactory.shardRecordProcessor(streamName), checkpoint, checkpointer, parentShardPollIntervalMillis, @@ -645,10 +659,10 @@ public class Scheduler implements Runnable { maxListShardsRetryAttempts, processorConfig.callProcessRecordsEvenForEmptyRecordList(), shardConsumerDispatchPollIntervalMillis, - initialPosition, + streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, - shardDetector, + shardDetectorProvider.apply(streamConfig.streamName()), aggregatorUtil, hierarchicalShardSyncer, metricsFactory); 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 3c71934b..6504bb24 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 @@ -268,36 +268,40 @@ public class LeaseManagementConfig { @Deprecated public LeaseManagementFactory leaseManagementFactory() { - Validate.notEmpty(streamName(), "Stream name is empty"); - return new DynamoDBLeaseManagementFactory(kinesisClient(), - streamName(), - dynamoDBClient(), - tableName(), - workerIdentifier(), - executorService(), - initialPositionInStream(), - failoverTimeMillis(), - epsilonMillis(), - maxLeasesForWorker(), - maxLeasesToStealAtOneTime(), - maxLeaseRenewalThreads(), - cleanupLeasesUponShardCompletion(), - ignoreUnexpectedChildShards(), - shardSyncIntervalMillis(), - consistentReads(), - listShardsBackoffTimeInMillis(), - maxListShardsRetryAttempts(), - maxCacheMissesBeforeReload(), - listShardsCacheAllowedAgeInSeconds(), - cacheMissWarningModulus(), - initialLeaseTableReadCapacity(), - initialLeaseTableWriteCapacity(), - hierarchicalShardSyncer(), - tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode()); + if (leaseManagementFactory == null) { + Validate.notEmpty(streamName(), "Stream name is empty"); + leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), + streamName(), + dynamoDBClient(), + tableName(), + workerIdentifier(), + executorService(), + initialPositionInStream(), + failoverTimeMillis(), + epsilonMillis(), + maxLeasesForWorker(), + maxLeasesToStealAtOneTime(), + maxLeaseRenewalThreads(), + cleanupLeasesUponShardCompletion(), + ignoreUnexpectedChildShards(), + shardSyncIntervalMillis(), + consistentReads(), + listShardsBackoffTimeInMillis(), + maxListShardsRetryAttempts(), + maxCacheMissesBeforeReload(), + listShardsCacheAllowedAgeInSeconds(), + cacheMissWarningModulus(), + initialLeaseTableReadCapacity(), + initialLeaseTableWriteCapacity(), + hierarchicalShardSyncer(), + tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode()); + } + return leaseManagementFactory; } public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer) { - return new DynamoDBLeaseManagementFactory(kinesisClient(), + if(leaseManagementFactory == null) { + leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), dynamoDBClient(), tableName(), workerIdentifier(), @@ -323,6 +327,13 @@ public class LeaseManagementConfig { dynamoDbRequestTimeout(), billingMode(), leaseSerializer); + } + return leaseManagementFactory; + } + + public LeaseManagementConfig leaseManagementFactory(final LeaseManagementFactory leaseManagementFactory) { + this.leaseManagementFactory = leaseManagementFactory; + return this; } // private InitialPositionInStreamExtended getInitialPositionExtendedForStream(String streamName) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 6a6c73b7..2d51c0bc 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -105,7 +105,7 @@ public class ShardInfo { @Override public int hashCode() { return new HashCodeBuilder() - .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamName).toHashCode(); + .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamName.orElse("")).toHashCode(); } /** @@ -130,7 +130,7 @@ public class ShardInfo { ShardInfo other = (ShardInfo) obj; return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken) .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId) - .append(streamName, other.streamName).isEquals(); + .append(streamName.orElse(""), other.streamName.orElse("")).isEquals(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java index 973b0393..4b691401 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java @@ -24,8 +24,7 @@ public interface ShardRecordProcessorFactory { * * @return */ - // TODO : Halo : Reenable -// ShardRecordProcessor shardRecordProcessor(); + ShardRecordProcessor shardRecordProcessor(); /** * Returns a new instance of the ShardRecordProcessor for a stream @@ -33,6 +32,6 @@ public interface ShardRecordProcessorFactory { * @return ShardRecordProcessor */ default ShardRecordProcessor shardRecordProcessor(String streamName) { - throw new UnsupportedOperationException(); + return shardRecordProcessor(); } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index 8be1bb8f..a17ad4cb 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 @@ -52,6 +52,7 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -498,6 +499,11 @@ public class SchedulerTest { } }; } + + @Override + public ShardRecordProcessor shardRecordProcessor(String streamName) { + return shardRecordProcessor(); + } } private class TestKinesisLeaseManagementFactory implements LeaseManagementFactory { @@ -511,6 +517,12 @@ public class SchedulerTest { return shardSyncTaskManager; } + @Override + public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, + StreamConfig streamConfig) { + return shardSyncTaskManager; + } + @Override public DynamoDBLeaseRefresher createLeaseRefresher() { return dynamoDBLeaseRefresher; @@ -520,6 +532,11 @@ public class SchedulerTest { public ShardDetector createShardDetector() { return shardDetector; } + + @Override + public ShardDetector createShardDetector(StreamConfig streamConfig) { + return shardDetector; + } } private class TestKinesisCheckpointFactory implements CheckpointFactory { From 9fa0cee97ed2f0b5f4ee0616e5b4218e5ac507f8 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 19 Feb 2020 18:43:45 -0800 Subject: [PATCH 009/159] ShardSyncer multistream changes --- .../amazon/kinesis/coordinator/Scheduler.java | 8 +- .../kinesis/leases/CompositeLeaseKey.java | 46 ----- .../leases/HierarchicalShardSyncer.java | 169 ++++++++++++++---- .../kinesis/leases/KinesisShardDetector.java | 2 +- .../kinesis/leases/LeaseManagementConfig.java | 20 ++- .../amazon/kinesis/leases/ShardDetector.java | 4 + .../dynamodb/DynamoDBLeaseCoordinator.java | 1 - 7 files changed, 162 insertions(+), 88 deletions(-) delete mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java 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 f564f47b..9ccf5df5 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 @@ -199,7 +199,8 @@ public class Scheduler implements Runnable { final LeaseSerializer leaseSerializer = this.appStreamTracker.map(mst -> true, sc -> false) ? new DynamoDBMultiStreamLeaseSerializer() : new DynamoDBLeaseSerializer(); - this.leaseCoordinator = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer) + this.leaseCoordinator = this.leaseManagementConfig + .leaseManagementFactory(leaseSerializer, this.appStreamTracker.map(mst -> true, sc -> false)) .createLeaseCoordinator(this.metricsFactory); this.leaseRefresher = this.leaseCoordinator.leaseRefresher(); @@ -220,7 +221,7 @@ public class Scheduler implements Runnable { // TODO : Halo : Handle case of no StreamConfig present in streamConfigMap() for the supplied streamName. // TODO : Pass the immutable map here instead of using mst.streamConfigMap() this.shardSyncTaskManagerProvider = streamName -> this.leaseManagementConfig - .leaseManagementFactory(leaseSerializer) + .leaseManagementFactory(leaseSerializer, this.appStreamTracker.map(mst -> true, sc -> false)) .createShardSyncTaskManager(this.metricsFactory, this.currentStreamConfigMap.get(streamName)); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); @@ -248,7 +249,8 @@ public class Scheduler implements Runnable { this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); // TODO : Halo : Check if this needs to be per stream. - this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(); + this.hierarchicalShardSyncer = leaseManagementConfig + .hierarchicalShardSyncer(this.appStreamTracker.map(mst -> true, sc -> false)); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java deleted file mode 100644 index 7092ceaa..00000000 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/CompositeLeaseKey.java +++ /dev/null @@ -1,46 +0,0 @@ -package software.amazon.kinesis.leases; - -import lombok.Getter; -import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.Validate; - -import java.util.Optional; - -public class CompositeLeaseKey { - -// private static final String LEASE_TOKEN_SEPERATOR = ":"; -// -// private String streamName; -// -// @Getter -// private String shardId; -// -// public CompositeLeaseKey(String shardId) { -// this(null, shardId); -// } -// -// public CompositeLeaseKey(String streamName, String shardId) { -// this.streamName = streamName; -// this.shardId = shardId; -// } -// -// public Optional getStreamName() { -// return Optional.ofNullable(streamName); -// } -// -// public String getLeaseKey(boolean isMultiStreamingEnabled) { -// Validate.isTrue(!(isMultiStreamingEnabled && StringUtils.isEmpty(streamName)), -// "Empty stream name found while multiStreaming is enabled"); -// return isMultiStreamingEnabled ? StringUtils.joinWith(LEASE_TOKEN_SEPERATOR, streamName, shardId) : shardId; -// } -// -// public static CompositeLeaseKey getLeaseKey(String leaseKey) { -// Validate.notNull(leaseKey); -// String leaseTokens[] = leaseKey.split(LEASE_TOKEN_SEPERATOR); -// Validate.inclusiveBetween(1, 2, leaseTokens.length); -// return leaseTokens.length == 2 ? -// new CompositeLeaseKey(leaseTokens[0], leaseTokens[1]) : -// new CompositeLeaseKey(leaseTokens[0]); -// } - -} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index 578af465..d3365514 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 @@ -25,9 +25,13 @@ import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; import lombok.NonNull; @@ -35,6 +39,7 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.awssdk.utils.Pair; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -57,6 +62,22 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @KinesisClientInternalApi public class HierarchicalShardSyncer { + private final boolean isMultiStreamMode; + + public HierarchicalShardSyncer() { + isMultiStreamMode = false; + } + + public HierarchicalShardSyncer(final boolean isMultiStreamMode) { + this.isMultiStreamMode = isMultiStreamMode; + } + + private static final BiFunction shardIdFromLeaseDeducer = + (lease, multiStreamArgs) -> + multiStreamArgs.isMultiStreamMode() ? + ((MultiStreamLease) lease).shardId() : + lease.leaseKey(); + /** * Check and create leases for any new shards (e.g. following a reshard operation). Sync leases with Kinesis shards * (e.g. at startup, or when we reach end of a shard). @@ -86,7 +107,8 @@ public class HierarchicalShardSyncer { //Provide a pre-collcted list of shards to avoid calling ListShards API public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards)throws DependencyException, InvalidStateException, + final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { if (!CollectionUtils.isNullOrEmpty(latestShards)) { log.debug("Num shards: {}", latestShards.size()); @@ -99,10 +121,12 @@ public class HierarchicalShardSyncer { if (!ignoreUnexpectedChildShards) { assertAllParentShardsAreClosed(inconsistentShardIds); } - - final List currentLeases = leaseRefresher.listLeases(); - - final List newLeasesToCreate = determineNewLeasesToCreate(latestShards, currentLeases, initialPosition, inconsistentShardIds); + final List currentLeases = isMultiStreamMode ? + getLeasesForStream(shardDetector.streamName(), leaseRefresher) : + leaseRefresher.listLeases(); + final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamName()); + final List newLeasesToCreate = determineNewLeasesToCreate(latestShards, currentLeases, initialPosition, + inconsistentShardIds, multiStreamArgs); log.debug("Num new leases to create: {}", newLeasesToCreate.size()); for (Lease lease : newLeasesToCreate) { long startTime = System.currentTimeMillis(); @@ -116,14 +140,36 @@ public class HierarchicalShardSyncer { } final List trackedLeases = new ArrayList<>(currentLeases); trackedLeases.addAll(newLeasesToCreate); - cleanupGarbageLeases(shardDetector, latestShards, trackedLeases, leaseRefresher); + cleanupGarbageLeases(shardDetector, latestShards, trackedLeases, leaseRefresher, multiStreamArgs); if (cleanupLeasesOfCompletedShards) { - cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases, leaseRefresher); + cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases, + leaseRefresher, multiStreamArgs); } - } + // CHECKSTYLE:ON CyclomaticComplexity + /** Note: This method has package level access solely for testing purposes. + * + * @param streamName We'll use this stream name to filter leases + * @param leaseRefresher Used to fetch leases + * @return Return list of leases (corresponding to shards) of the specified stream. + * @throws DependencyException + * @throws InvalidStateException + * @throws ProvisionedThroughputException + */ + static List getLeasesForStream(String streamName, + LeaseRefresher leaseRefresher) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamLeases = new ArrayList<>(); + for (Lease lease : leaseRefresher.listLeases()) { + if (streamName.equals(((MultiStreamLease)lease).streamName())) { + streamLeases.add(lease); + } + } + return streamLeases; + } + /** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls * and a reshard operation. * @param inconsistentShardIds @@ -310,12 +356,15 @@ public class HierarchicalShardSyncer { * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ static List determineNewLeasesToCreate(final List shards, final List currentLeases, - final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds) { + final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds, + final MultiStreamArgs multiStreamArgs) { final Map shardIdToNewLeaseMap = new HashMap<>(); final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); final Set shardIdsOfCurrentLeases = currentLeases.stream() - .peek(lease -> log.debug("Existing lease: {}", lease)).map(Lease::leaseKey).collect(Collectors.toSet()); + .peek(lease -> log.debug("Existing lease: {}", lease)) + .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .collect(Collectors.toSet()); final List openShards = getOpenShards(shards); final Map memoizationContext = new HashMap<>(); @@ -330,10 +379,12 @@ public class HierarchicalShardSyncer { log.info("shardId {} is an inconsistent child. Not creating a lease", shardId); } else { log.debug("Need to create a lease for shardId {}", shardId); - final Lease newLease = newKCLLease(shard); + final Lease newLease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shard, multiStreamArgs.streamName()) : + newKCLLease(shard); final boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, - memoizationContext); + memoizationContext, multiStreamArgs); /** * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the @@ -371,11 +422,17 @@ public class HierarchicalShardSyncer { final List newLeasesToCreate = new ArrayList<>(shardIdToNewLeaseMap.values()); final Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMapOfAllKinesisShards); + shardIdToShardMapOfAllKinesisShards, multiStreamArgs); newLeasesToCreate.sort(startingSequenceNumberComparator); return newLeasesToCreate; } + static List determineNewLeasesToCreate(final List shards, final List currentLeases, + final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds) { + return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds, + new MultiStreamArgs(false, null)); + } + /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. @@ -405,7 +462,8 @@ public class HierarchicalShardSyncer { static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, final Map shardIdToShardMapOfAllKinesisShards, - final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext) { + final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext, + final MultiStreamArgs multiStreamArgs) { final Boolean previousValue = memoizationContext.get(shardId); if (previousValue != null) { @@ -428,7 +486,7 @@ public class HierarchicalShardSyncer { // Check if the parent is a descendant, and include its ancestors. if (checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, - memoizationContext)) { + memoizationContext, multiStreamArgs)) { isDescendant = true; descendantParentShardIds.add(parentShardId); log.debug("Parent shard {} is a descendant.", parentShardId); @@ -444,7 +502,10 @@ public class HierarchicalShardSyncer { log.debug("Need to create a lease for shardId {}", parentShardId); Lease lease = shardIdToLeaseMapOfNewShards.get(parentShardId); if (lease == null) { - lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); + lease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId), + multiStreamArgs.streamName()) : + newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); shardIdToLeaseMapOfNewShards.put(parentShardId, lease); } @@ -475,6 +536,16 @@ public class HierarchicalShardSyncer { memoizationContext.put(shardId, isDescendant); return isDescendant; } + + static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, + final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, + final Map shardIdToShardMapOfAllKinesisShards, + final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext) { + return checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, + shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, memoizationContext, + new MultiStreamArgs(false, null)); + } + // CHECKSTYLE:ON CyclomaticComplexity /** @@ -519,13 +590,14 @@ public class HierarchicalShardSyncer { * @throws DependencyException */ private static void cleanupGarbageLeases(@NonNull final ShardDetector shardDetector, final List shards, - final List trackedLeases, final LeaseRefresher leaseRefresher) throws KinesisClientLibIOException, + final List trackedLeases, final LeaseRefresher leaseRefresher, + final MultiStreamArgs multiStreamArgs) throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException { final Set kinesisShards = shards.stream().map(Shard::shardId).collect(Collectors.toSet()); // Check if there are leases for non-existent shards final List garbageLeases = trackedLeases.stream() - .filter(lease -> isCandidateForCleanup(lease, kinesisShards)).collect(Collectors.toList()); + .filter(lease -> isCandidateForCleanup(lease, kinesisShards, multiStreamArgs)).collect(Collectors.toList()); if (!CollectionUtils.isNullOrEmpty(garbageLeases)) { log.info("Found {} candidate leases for cleanup. Refreshing list of" @@ -534,7 +606,7 @@ public class HierarchicalShardSyncer { .collect(Collectors.toSet()); for (Lease lease : garbageLeases) { - if (isCandidateForCleanup(lease, currentKinesisShardIds)) { + if (isCandidateForCleanup(lease, currentKinesisShardIds, multiStreamArgs)) { log.info("Deleting lease for shard {} as it is not present in Kinesis stream.", lease.leaseKey()); leaseRefresher.deleteLease(lease); } @@ -552,14 +624,17 @@ public class HierarchicalShardSyncer { * @throws KinesisClientLibIOException Thrown if currentKinesisShardIds contains a parent shard but not the child * shard (we are evaluating for deletion). */ - static boolean isCandidateForCleanup(final Lease lease, final Set currentKinesisShardIds) + static boolean isCandidateForCleanup(final Lease lease, final Set currentKinesisShardIds, + final MultiStreamArgs multiStreamArgs) throws KinesisClientLibIOException { boolean isCandidateForCleanup = true; - if (currentKinesisShardIds.contains(lease.leaseKey())) { + final String shardId = shardIdFromLeaseDeducer.apply(lease, multiStreamArgs); + + if (currentKinesisShardIds.contains(shardId)) { isCandidateForCleanup = false; } else { - log.info("Found lease for non-existent shard: {}. Checking its parent shards", lease.leaseKey()); + log.info("Found lease for non-existent shard: {}. Checking its parent shards", shardId); final Set parentShardIds = lease.parentShardIds(); for (String parentShardId : parentShardIds) { @@ -567,7 +642,7 @@ public class HierarchicalShardSyncer { // This may be a (rare) race condition between fetching the shard list and Kinesis expiring shards. if (currentKinesisShardIds.contains(parentShardId)) { final String message = String.format("Parent shard %s exists but not the child shard %s", - parentShardId, lease.leaseKey()); + parentShardId, shardId); log.info(message); throw new KinesisClientLibIOException(message); } @@ -596,27 +671,28 @@ public class HierarchicalShardSyncer { */ private synchronized void cleanupLeasesOfFinishedShards(final Collection currentLeases, final Map shardIdToShardMap, final Map> shardIdToChildShardIdsMap, - final List trackedLeases, final LeaseRefresher leaseRefresher) throws DependencyException, + final List trackedLeases, final LeaseRefresher leaseRefresher, + final MultiStreamArgs multiStreamArgs) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { final List leasesOfClosedShards = currentLeases.stream() .filter(lease -> lease.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) .collect(Collectors.toList()); - final Set shardIdsOfClosedShards = leasesOfClosedShards.stream().map(Lease::leaseKey) - .collect(Collectors.toSet()); + final Set shardIdsOfClosedShards = leasesOfClosedShards.stream() + .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)).collect(Collectors.toSet()); if (!CollectionUtils.isNullOrEmpty(leasesOfClosedShards)) { assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, shardIdsOfClosedShards); Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMap); + shardIdToShardMap, multiStreamArgs); leasesOfClosedShards.sort(startingSequenceNumberComparator); final Map trackedLeaseMap = trackedLeases.stream() - .collect(Collectors.toMap(Lease::leaseKey, Function.identity())); + .collect(Collectors.toMap(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs), Function.identity())); for (Lease leaseOfClosedShard : leasesOfClosedShards) { - final String closedShardId = leaseOfClosedShard.leaseKey(); + final String closedShardId = shardIdFromLeaseDeducer.apply(leaseOfClosedShard, multiStreamArgs); final Set childShardIds = shardIdToChildShardIdsMap.get(closedShardId); if (closedShardId != null && !CollectionUtils.isNullOrEmpty(childShardIds)) { - cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseRefresher); + cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseRefresher, multiStreamArgs); } } } @@ -637,7 +713,7 @@ public class HierarchicalShardSyncer { * @throws DependencyException */ synchronized void cleanupLeaseForClosedShard(final String closedShardId, final Set childShardIds, - final Map trackedLeases, final LeaseRefresher leaseRefresher) + final Map trackedLeases, final LeaseRefresher leaseRefresher, final MultiStreamArgs multiStreamArgs) throws DependencyException, InvalidStateException, ProvisionedThroughputException { final Lease leaseForClosedShard = trackedLeases.get(closedShardId); final List childShardLeases = childShardIds.stream().map(trackedLeases::get).filter(Objects::nonNull) @@ -655,7 +731,7 @@ public class HierarchicalShardSyncer { if (okayToDelete) { log.info("Deleting lease for shard {} as it has been completely processed and processing of child " - + "shards has begun.", leaseForClosedShard.leaseKey()); + + "shards has begun.", shardIdFromLeaseDeducer.apply(leaseForClosedShard, multiStreamArgs)); leaseRefresher.deleteLease(leaseForClosedShard); } } @@ -684,6 +760,23 @@ public class HierarchicalShardSyncer { return newLease; } + private static Lease newKCLMultiStreamLease(final Shard shard, final String streamName) { + MultiStreamLease newLease = new MultiStreamLease(); + newLease.leaseKey(MultiStreamLease.getLeaseKey(streamName, shard.shardId())); + List parentShardIds = new ArrayList<>(2); + if (shard.parentShardId() != null) { + parentShardIds.add(shard.parentShardId()); + } + if (shard.adjacentParentShardId() != null) { + parentShardIds.add(shard.adjacentParentShardId()); + } + newLease.parentShardIds(parentShardIds); + newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.streamName(streamName); + newLease.shardId(shard.shardId()); + return newLease; + } + /** * Helper method to construct a shardId->Shard map for the specified list of shards. * @@ -728,6 +821,7 @@ public class HierarchicalShardSyncer { private static final long serialVersionUID = 1L; private final Map shardIdToShardMap; + private final MultiStreamArgs multiStreamArgs; /** * Compares two leases based on the starting sequence number of corresponding shards. @@ -741,8 +835,8 @@ public class HierarchicalShardSyncer { @Override public int compare(final Lease lease1, final Lease lease2) { int result = 0; - final String shardId1 = lease1.leaseKey(); - final String shardId2 = lease2.leaseKey(); + final String shardId1 = shardIdFromLeaseDeducer.apply(lease1, multiStreamArgs); + final String shardId2 = shardIdFromLeaseDeducer.apply(lease2, multiStreamArgs); final Shard shard1 = shardIdToShardMap.get(shardId1); final Shard shard2 = shardIdToShardMap.get(shardId2); @@ -762,4 +856,11 @@ public class HierarchicalShardSyncer { } + @Data + @Accessors(fluent = true) + private static class MultiStreamArgs { + private final Boolean isMultiStreamMode; + private final String streamName; + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java index ba136f0a..f009a0e7 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 @@ -59,7 +59,7 @@ public class KinesisShardDetector implements ShardDetector { @NonNull private final KinesisAsyncClient kinesisClient; - @NonNull + @NonNull @Getter private final String streamName; private final long listShardsBackoffTimeInMillis; private final int maxListShardsRetryAttempts; 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 6504bb24..3361e3fb 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 @@ -262,10 +262,24 @@ public class LeaseManagementConfig { */ private TableCreatorCallback tableCreatorCallback = TableCreatorCallback.NOOP_TABLE_CREATOR_CALLBACK; - private HierarchicalShardSyncer hierarchicalShardSyncer = new HierarchicalShardSyncer(); + private HierarchicalShardSyncer hierarchicalShardSyncer; private LeaseManagementFactory leaseManagementFactory; + private HierarchicalShardSyncer hierarchicalShardSyncer() { + if(hierarchicalShardSyncer == null) { + hierarchicalShardSyncer = new HierarchicalShardSyncer(); + } + return hierarchicalShardSyncer; + } + + public HierarchicalShardSyncer hierarchicalShardSyncer(boolean isMultiStreamingMode) { + if(hierarchicalShardSyncer == null) { + hierarchicalShardSyncer = new HierarchicalShardSyncer(isMultiStreamingMode); + } + return hierarchicalShardSyncer; + } + @Deprecated public LeaseManagementFactory leaseManagementFactory() { if (leaseManagementFactory == null) { @@ -299,7 +313,7 @@ public class LeaseManagementConfig { return leaseManagementFactory; } - public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer) { + public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer, boolean isMultiStreamingMode) { if(leaseManagementFactory == null) { leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), dynamoDBClient(), @@ -322,7 +336,7 @@ public class LeaseManagementConfig { cacheMissWarningModulus(), initialLeaseTableReadCapacity(), initialLeaseTableWriteCapacity(), - hierarchicalShardSyncer(), + hierarchicalShardSyncer(isMultiStreamingMode), tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode(), diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java index cf3a1a78..43e1e1b3 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 @@ -27,4 +27,8 @@ public interface ShardDetector { List listShards(); + default String streamName() { + throw new UnsupportedOperationException("StreamName not available"); + } + } 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 2252da84..c0d3913b 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 @@ -33,7 +33,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import lombok.extern.slf4j.Slf4j; import software.amazon.kinesis.annotations.KinesisClientInternalApi; -import software.amazon.kinesis.leases.CompositeLeaseKey; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; From 9ad65ee4868781837b2e30aa24fd82834b2039b3 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 20 Feb 2020 00:24:09 -0800 Subject: [PATCH 010/159] Optional bug fix --- .../java/software/amazon/kinesis/coordinator/Scheduler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 9ccf5df5..0ba40d81 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 @@ -367,7 +367,7 @@ public class Scheduler implements Runnable { for (ShardInfo completedShard : completedShards) { final String streamName = completedShard.streamName() - .orElse(appStreamTracker.map(mst -> null, sc -> sc.streamName())); + .orElseGet(() -> appStreamTracker.map(mst -> null, sc -> sc.streamName())); Validate.notEmpty(streamName, "Stream name should not be null"); if (createOrGetShardSyncTaskManager(streamName).syncShardAndLeaseInfo()) { log.info("Found completed shard, initiated new ShardSyncTak for " + completedShard.toString()); @@ -640,7 +640,7 @@ public class Scheduler implements Runnable { checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. - final String streamName = shardInfo.streamName().orElse(appStreamTracker.map(mst -> null, sc -> sc.streamName())); + final String streamName = shardInfo.streamName().orElseGet(() -> appStreamTracker.map(mst -> null, sc -> sc.streamName())); Validate.notEmpty(streamName, "StreamName should not be empty"); // Irrespective of single stream app or multi stream app, streamConfig should always be available. // TODO: Halo : if not available, construct a default config ? From 097559eca2fe3466bcc65d1d8d31791b58ff17f7 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 20 Feb 2020 01:55:31 -0800 Subject: [PATCH 011/159] Checkpointing and retrieval fix --- .../checkpoint/ShardRecordProcessorCheckpointer.java | 12 ++++++------ .../amazon/kinesis/coordinator/Scheduler.java | 6 +++--- .../software/amazon/kinesis/leases/ShardInfo.java | 11 +++++++++++ .../amazon/kinesis/lifecycle/InitializeTask.java | 6 ++++-- .../kinesis/retrieval/fanout/FanOutConfig.java | 10 +++++----- .../retrieval/fanout/FanOutRetrievalFactory.java | 8 ++++++-- .../polling/SynchronousBlockingRetrievalFactory.java | 2 +- .../SynchronousPrefetchingRetrievalFactory.java | 8 ++++---- 8 files changed, 40 insertions(+), 23 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 4705d564..7d504bbb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java @@ -60,7 +60,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi public synchronized void checkpoint() throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { if (log.isDebugEnabled()) { - log.debug("Checkpointing {}, token {} at largest permitted value {}", shardInfo.shardId(), + log.debug("Checkpointing {}, token {} at largest permitted value {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), this.largestPermittedCheckpointValue); } advancePosition(this.largestPermittedCheckpointValue); @@ -116,7 +116,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi && newCheckpoint.compareTo(largestPermittedCheckpointValue) <= 0) { if (log.isDebugEnabled()) { - log.debug("Checkpointing {}, token {} at specific extended sequence number {}", shardInfo.shardId(), + log.debug("Checkpointing {}, token {} at specific extended sequence number {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), newCheckpoint); } this.advancePosition(newCheckpoint); @@ -189,7 +189,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi if (log.isDebugEnabled()) { log.debug("Preparing checkpoint {}, token {} at specific extended sequence number {}", - shardInfo.shardId(), shardInfo.concurrencyToken(), pendingCheckpoint); + ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), pendingCheckpoint); } return doPrepareCheckpoint(pendingCheckpoint); } else { @@ -252,10 +252,10 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi if (extendedSequenceNumber != null && !extendedSequenceNumber.equals(lastCheckpointValue)) { try { if (log.isDebugEnabled()) { - log.debug("Setting {}, token {} checkpoint to {}", shardInfo.shardId(), + log.debug("Setting {}, token {} checkpoint to {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), checkpointToRecord); } - checkpointer.setCheckpoint(shardInfo.shardId(), checkpointToRecord, shardInfo.concurrencyToken()); + checkpointer.setCheckpoint(ShardInfo.getLeaseKey(shardInfo), checkpointToRecord, shardInfo.concurrencyToken()); lastCheckpointValue = checkpointToRecord; } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { @@ -308,7 +308,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } try { - checkpointer.prepareCheckpoint(shardInfo.shardId(), newPrepareCheckpoint, shardInfo.concurrencyToken()); + checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken()); } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { throw e; 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 0ba40d81..547feebb 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 @@ -367,8 +367,8 @@ public class Scheduler implements Runnable { for (ShardInfo completedShard : completedShards) { final String streamName = completedShard.streamName() - .orElseGet(() -> appStreamTracker.map(mst -> null, sc -> sc.streamName())); - Validate.notEmpty(streamName, "Stream name should not be null"); + .orElseGet(() -> appStreamTracker.map(mst -> "", sc -> sc.streamName())); + Validate.notEmpty(streamName, "Stream name should not be empty"); if (createOrGetShardSyncTaskManager(streamName).syncShardAndLeaseInfo()) { log.info("Found completed shard, initiated new ShardSyncTak for " + completedShard.toString()); } @@ -640,7 +640,7 @@ public class Scheduler implements Runnable { checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. - final String streamName = shardInfo.streamName().orElseGet(() -> appStreamTracker.map(mst -> null, sc -> sc.streamName())); + final String streamName = shardInfo.streamName().orElseGet(() -> appStreamTracker.map(mst -> "", sc -> sc.streamName())); Validate.notEmpty(streamName, "StreamName should not be empty"); // Irrespective of single stream app or multi stream app, streamConfig should always be available. // TODO: Halo : if not available, construct a default config ? diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 2d51c0bc..0f86efb2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -134,4 +134,15 @@ public class ShardInfo { } + /** + * + * @param shardInfo + * @return + */ + public static String getLeaseKey(ShardInfo shardInfo) { + return shardInfo.streamName().isPresent() ? + MultiStreamLease.getLeaseKey(shardInfo.streamName().get(), shardInfo.shardId()) : + shardInfo.shardId(); + } + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java index fdb0e947..e11eebfa 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java @@ -21,6 +21,7 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.metrics.MetricsFactory; @@ -75,9 +76,10 @@ public class InitializeTask implements ConsumerTask { try { log.debug("Initializing ShardId {}", shardInfo); - Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(shardInfo.shardId()); + final String leaseKey = ShardInfo.getLeaseKey(shardInfo); + Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(leaseKey); ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.checkpoint(); - log.debug("[{}]: Checkpoint: {} -- Initial Position: {}", shardInfo.shardId(), initialCheckpoint, + log.debug("[{}]: Checkpoint: {} -- Initial Position: {}", leaseKey, initialCheckpoint, initialPositionInStream); cache.start(initialCheckpoint, initialPositionInStream); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java index 661c2841..45679089 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 @@ -82,15 +82,15 @@ public class FanOutConfig implements RetrievalSpecificConfig { @Override public RetrievalFactory retrievalFactory() { - return new FanOutRetrievalFactory(kinesisClient, getOrCreateConsumerArn()); + return new FanOutRetrievalFactory(kinesisClient, streamName, this::getOrCreateConsumerArn); } - private String getOrCreateConsumerArn() { + private String getOrCreateConsumerArn(String streamName) { if (consumerArn != null) { return consumerArn; } - FanOutConsumerRegistration registration = createConsumerRegistration(); + FanOutConsumerRegistration registration = createConsumerRegistration(streamName); try { return registration.getOrCreateStreamConsumerArn(); } catch (DependencyException e) { @@ -98,10 +98,10 @@ public class FanOutConfig implements RetrievalSpecificConfig { } } - private FanOutConsumerRegistration createConsumerRegistration() { + private FanOutConsumerRegistration createConsumerRegistration(String streamName) { String consumerToCreate = ObjectUtils.firstNonNull(consumerName(), applicationName()); return createConsumerRegistration(kinesisClient(), - Preconditions.checkNotNull(streamName(), "streamName must be set for consumer creation"), + Preconditions.checkNotNull(streamName, "streamName must be set for consumer creation"), Preconditions.checkNotNull(consumerToCreate, "applicationName or consumerName must be set for consumer creation")); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java index 4add0dab..e712c6db 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 @@ -25,12 +25,15 @@ import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; +import java.util.function.Function; + @RequiredArgsConstructor @KinesisClientInternalApi public class FanOutRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; - private final String consumerArn; + private final String defaultStreamName; + private final Function consumerArnProvider; @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo, @@ -41,6 +44,7 @@ public class FanOutRetrievalFactory implements RetrievalFactory { @Override public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, final MetricsFactory metricsFactory) { - return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), consumerArn); + final String streamName = shardInfo.streamName().orElse(defaultStreamName); + return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), consumerArnProvider.apply(streamName)); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java index ac40c7d2..7405730e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java @@ -63,7 +63,7 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory { public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, streamName, shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); + new KinesisDataFetcher(kinesisClient, shardInfo.streamName().orElse(streamName), shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java index 320fe4dd..8b669893 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java @@ -65,11 +65,11 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory this.maxFutureWait = maxFutureWait; } - @Override - public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, + @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { - return new SynchronousGetRecordsRetrievalStrategy(new KinesisDataFetcher(kinesisClient, streamName, - shardInfo.shardId(), maxRecords, metricsFactory, maxFutureWait)); + return new SynchronousGetRecordsRetrievalStrategy( + new KinesisDataFetcher(kinesisClient, shardInfo.streamName().orElse(streamName), shardInfo.shardId(), + maxRecords, metricsFactory, maxFutureWait)); } @Override From fedd02c2a5b2dbc27d156e565c15792fc5d5db22 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 20 Feb 2020 15:48:56 -0800 Subject: [PATCH 012/159] Fixing fanout cross consumer issue --- .../kinesis/retrieval/fanout/FanOutRetrievalFactory.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 e712c6db..5c0f5e9a 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 @@ -25,6 +25,8 @@ import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; +import java.util.HashMap; +import java.util.Map; import java.util.function.Function; @RequiredArgsConstructor @@ -34,6 +36,7 @@ public class FanOutRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; private final String defaultStreamName; private final Function consumerArnProvider; + private Map streamToconsumerArnMap = new HashMap<>(); @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo, @@ -45,6 +48,7 @@ public class FanOutRetrievalFactory implements RetrievalFactory { public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, final MetricsFactory metricsFactory) { final String streamName = shardInfo.streamName().orElse(defaultStreamName); - return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), consumerArnProvider.apply(streamName)); + return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), + streamToconsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); } } From 2b507342d89f6a7a031f9330ab30a46c5fa98fef Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 3 Mar 2020 16:07:07 -0800 Subject: [PATCH 013/159] Adding account and stream epoch support. Checkpoint 1 --- .../MultiLangDaemonConfigurationTest.java | 8 +++ .../amazon/kinesis/common/StreamConfig.java | 4 +- .../kinesis/common/StreamIdentifier.java | 33 +++++++++ .../amazon/kinesis/coordinator/Scheduler.java | 69 +++++++++++-------- .../leases/HierarchicalShardSyncer.java | 6 +- .../kinesis/leases/MultiStreamLease.java | 14 ++-- .../amazon/kinesis/leases/ShardInfo.java | 14 ++-- .../dynamodb/DynamoDBLeaseCoordinator.java | 2 +- .../DynamoDBMultiStreamLeaseSerializer.java | 6 +- .../lifecycle/ShardConsumerArgument.java | 4 +- .../kinesis/processor/MultiStreamTracker.java | 3 +- .../ShardRecordProcessorFactory.java | 6 +- .../fanout/FanOutRetrievalFactory.java | 2 +- .../SynchronousBlockingRetrievalFactory.java | 2 +- ...ynchronousPrefetchingRetrievalFactory.java | 2 +- 15 files changed, 115 insertions(+), 60 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java index 5101243b..07f8082b 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/config/MultiLangDaemonConfigurationTest.java @@ -171,6 +171,14 @@ public class MultiLangDaemonConfigurationTest { utilsBean.setProperty(configuration, "retrievalMode", "invalid"); } + // @Test + // TODO : Enable this test once https://github.com/awslabs/amazon-kinesis-client/issues/692 is resolved + public void testmetricsEnabledDimensions() { + MultiLangDaemonConfiguration configuration = baseConfiguration(); + configuration.setMetricsEnabledDimensions(new String[]{"Operation"}); + configuration.resolvedConfiguration(shardRecordProcessorFactory); + } + @Test public void testFanoutConfigSetConsumerName() { String consumerArn = "test-consumer"; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java index 55451709..667f1f1c 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 @@ -9,6 +9,8 @@ import lombok.experimental.FieldDefaults; @Accessors(fluent = true) @FieldDefaults(makeFinal=true, level= AccessLevel.PRIVATE) public class StreamConfig { - String streamName; + StreamIdentifier streamIdentifier; InitialPositionInStreamExtended initialPositionInStreamExtended; } + + diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java new file mode 100644 index 00000000..e54c97e6 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java @@ -0,0 +1,33 @@ +package software.amazon.kinesis.common; + +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import software.amazon.awssdk.utils.Validate; + +@RequiredArgsConstructor +@EqualsAndHashCode +@Getter +public class StreamIdentifier { + private final String accountName; + private final String streamName; + private final String streamCreationEpoch; + + private static final String DEFAULT = "default"; + + @Override + public String toString(){ + return accountName + ":" + streamName + ":" + streamCreationEpoch; + } + + public static StreamIdentifier fromString(String streamIdentifier) { + final String[] idTokens = streamIdentifier.split(":"); + Validate.isTrue(idTokens.length == 3, "Unable to deserialize StreamIdentifier from " + streamIdentifier); + return new StreamIdentifier(idTokens[0], idTokens[1], idTokens[2]); + } + + public static StreamIdentifier fromStreamName(String streamName) { + Validate.notEmpty(streamName, "StreamName should not be empty"); + return new StreamIdentifier(DEFAULT, streamName, DEFAULT); + } +} 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 547feebb..c94981b9 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 @@ -20,6 +20,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -45,6 +46,7 @@ import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -110,8 +112,8 @@ public class Scheduler implements Runnable { private final DiagnosticEventHandler diagnosticEventHandler; // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; - private final Function shardSyncTaskManagerProvider; - private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); + private final Function shardSyncTaskManagerProvider; + private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; @@ -121,11 +123,11 @@ public class Scheduler implements Runnable { private final long failoverTimeMillis; private final long taskBackoffTimeMillis; private final Either appStreamTracker; - private final Map currentStreamConfigMap; + private final Map currentStreamConfigMap; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; - private final Function shardDetectorProvider; + private final Function shardDetectorProvider; private final boolean ignoreUnexpetedChildShards; private final AggregatorUtil aggregatorUtil; private final HierarchicalShardSyncer hierarchicalShardSyncer; @@ -183,15 +185,16 @@ public class Scheduler implements Runnable { this.applicationName = this.coordinatorConfig.applicationName(); final MultiStreamTracker multiStreamTracker = this.retrievalConfig.multiStreamTracker(); if(multiStreamTracker == null) { - final StreamConfig streamConfig = new StreamConfig(this.retrievalConfig.streamName(), + final StreamConfig streamConfig = new StreamConfig(StreamIdentifier.fromStreamName(this.retrievalConfig.streamName()), this.retrievalConfig.initialPositionInStreamExtended()); this.appStreamTracker = Either.right(streamConfig); - this.currentStreamConfigMap = new HashMap() {{ - put(streamConfig.streamName(), streamConfig); + this.currentStreamConfigMap = new HashMap() {{ + put(streamConfig.streamIdentifier(), streamConfig); }}; } else { this.appStreamTracker = Either.left(multiStreamTracker); - this.currentStreamConfigMap = multiStreamTracker.streamConfigMap(); + this.currentStreamConfigMap = multiStreamTracker.streamConfigList().stream() + .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); } this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); @@ -218,11 +221,11 @@ public class Scheduler implements Runnable { this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService(); this.diagnosticEventFactory = diagnosticEventFactory; this.diagnosticEventHandler = new DiagnosticEventLogger(); - // TODO : Halo : Handle case of no StreamConfig present in streamConfigMap() for the supplied streamName. - // TODO : Pass the immutable map here instead of using mst.streamConfigMap() - this.shardSyncTaskManagerProvider = streamName -> this.leaseManagementConfig + // TODO : Halo : Handle case of no StreamConfig present in streamConfigList() for the supplied streamName. + // TODO : Pass the immutable map here instead of using mst.streamConfigList() + this.shardSyncTaskManagerProvider = streamIdentifier -> this.leaseManagementConfig .leaseManagementFactory(leaseSerializer, this.appStreamTracker.map(mst -> true, sc -> false)) - .createShardSyncTaskManager(this.metricsFactory, this.currentStreamConfigMap.get(streamName)); + .createShardSyncTaskManager(this.metricsFactory, this.currentStreamConfigMap.get(streamIdentifier)); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -245,7 +248,7 @@ public class Scheduler implements Runnable { // this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool(); this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis(); this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts(); - this.shardDetectorProvider = streamName -> createOrGetShardSyncTaskManager(streamName).shardDetector(); + this.shardDetectorProvider = streamIdentifier -> createOrGetShardSyncTaskManager(streamIdentifier).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); // TODO : Halo : Check if this needs to be per stream. @@ -298,17 +301,17 @@ public class Scheduler implements Runnable { if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - for(String streamName : currentStreamConfigMap.keySet().stream().collect(Collectors.toList())) { + for(StreamIdentifier streamIdentifier : currentStreamConfigMap.keySet().stream().collect(Collectors.toList())) { log.info("Syncing Kinesis shard info"); - final StreamConfig streamConfig = currentStreamConfigMap.get(streamName); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamName), + final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); + ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamIdentifier), leaseRefresher, streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, metricsFactory); result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); // Throwing the exception, to prevent further syncs for other stream. if (result.getException() != null) { - log.error("Caught exception when sync'ing info for " + streamName, result.getException()); + log.error("Caught exception when sync'ing info for " + streamIdentifier, result.getException()); throw result.getException(); } } @@ -366,10 +369,8 @@ public class Scheduler implements Runnable { } for (ShardInfo completedShard : completedShards) { - final String streamName = completedShard.streamName() - .orElseGet(() -> appStreamTracker.map(mst -> "", sc -> sc.streamName())); - Validate.notEmpty(streamName, "Stream name should not be empty"); - if (createOrGetShardSyncTaskManager(streamName).syncShardAndLeaseInfo()) { + final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifier()); + if (createOrGetShardSyncTaskManager(streamIdentifier).syncShardAndLeaseInfo()) { log.info("Found completed shard, initiated new ShardSyncTak for " + completedShard.toString()); } } @@ -629,8 +630,8 @@ public class Scheduler implements Runnable { return consumer; } - private ShardSyncTaskManager createOrGetShardSyncTaskManager(String streamName) { - return streamToShardSyncTaskManagerMap.computeIfAbsent(streamName, s -> shardSyncTaskManagerProvider.apply(s)); + private ShardSyncTaskManager createOrGetShardSyncTaskManager(StreamIdentifier streamIdentifier) { + return streamToShardSyncTaskManagerMap.computeIfAbsent(streamIdentifier, s -> shardSyncTaskManagerProvider.apply(s)); } protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, @@ -640,18 +641,17 @@ public class Scheduler implements Runnable { checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. - final String streamName = shardInfo.streamName().orElseGet(() -> appStreamTracker.map(mst -> "", sc -> sc.streamName())); - Validate.notEmpty(streamName, "StreamName should not be empty"); + final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifier()); // Irrespective of single stream app or multi stream app, streamConfig should always be available. // TODO: Halo : if not available, construct a default config ? - final StreamConfig streamConfig = currentStreamConfigMap.get(streamName); + final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); Validate.notNull(streamConfig, "StreamConfig should not be empty"); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, - streamConfig.streamName(), + streamConfig.streamIdentifier(), leaseCoordinator, executorService, cache, - shardRecordProcessorFactory.shardRecordProcessor(streamName), + shardRecordProcessorFactory.shardRecordProcessor(streamIdentifier), checkpoint, checkpointer, parentShardPollIntervalMillis, @@ -664,7 +664,7 @@ public class Scheduler implements Runnable { streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, - shardDetectorProvider.apply(streamConfig.streamName()), + shardDetectorProvider.apply(streamConfig.streamIdentifier()), aggregatorUtil, hierarchicalShardSyncer, metricsFactory); @@ -716,6 +716,17 @@ public class Scheduler implements Runnable { executorStateEvent.accept(diagnosticEventHandler); } + private StreamIdentifier getStreamIdentifier(Optional streamIdentifierString) { + final StreamIdentifier streamIdentifier; + if(streamIdentifierString.isPresent()) { + streamIdentifier = StreamIdentifier.fromString(streamIdentifierString.get()); + } else { + streamIdentifier = appStreamTracker.map(mst -> null, sc -> sc.streamIdentifier()); + } + Validate.notNull(streamIdentifier, "Stream identifier should not be empty"); + return streamIdentifier; + } + /** * Logger for suppressing too much INFO logging. To avoid too much logging information Worker will output logging at * INFO level for a single pass through the main loop every minute. At DEBUG level it will output all INFO logs on diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index d3365514..459d730d 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 @@ -29,7 +29,6 @@ import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; -import lombok.AllArgsConstructor; import lombok.Data; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; @@ -39,7 +38,6 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.utils.CollectionUtils; -import software.amazon.awssdk.utils.Pair; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -163,7 +161,7 @@ public class HierarchicalShardSyncer { throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamLeases = new ArrayList<>(); for (Lease lease : leaseRefresher.listLeases()) { - if (streamName.equals(((MultiStreamLease)lease).streamName())) { + if (streamName.equals(((MultiStreamLease)lease).streamIdentifier())) { streamLeases.add(lease); } } @@ -772,7 +770,7 @@ public class HierarchicalShardSyncer { } newLease.parentShardIds(parentShardIds); newLease.ownerSwitchesSinceCheckpoint(0L); - newLease.streamName(streamName); + newLease.streamIdentifier(streamName); newLease.shardId(shard.shardId()); return newLease; } 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 9878e32c..8b29168d 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 @@ -17,13 +17,13 @@ import static com.google.common.base.Verify.verifyNotNull; @Accessors(fluent = true) public class MultiStreamLease extends Lease { - @NonNull private String streamName; + @NonNull private String streamIdentifier; @NonNull private String shardId; public MultiStreamLease(Lease other) { super(other); MultiStreamLease casted = validateAndCast(other); - streamName(casted.streamName); + streamIdentifier(casted.streamIdentifier); shardId(casted.shardId); } @@ -31,7 +31,7 @@ public class MultiStreamLease extends Lease { public void update(Lease other) { MultiStreamLease casted = validateAndCast(other); super.update(casted); - streamName(casted.streamName); + streamIdentifier(casted.streamIdentifier); shardId(casted.shardId); } @@ -43,7 +43,7 @@ public class MultiStreamLease extends Lease { @Override public int hashCode() { - return Objects.hash(super.hashCode(), streamName); + return Objects.hash(super.hashCode(), streamIdentifier); } @Override @@ -58,11 +58,11 @@ public class MultiStreamLease extends Lease { return false; } MultiStreamLease other = (MultiStreamLease) obj; - if (streamName == null) { - if (other.streamName != null) { + if (streamIdentifier == null) { + if (other.streamIdentifier != null) { return false; } - } else if (!streamName.equals(other.streamName)) { + } else if (!streamIdentifier.equals(other.streamIdentifier)) { return false; } return true; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 0f86efb2..36bc5dd1 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -37,7 +37,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @ToString public class ShardInfo { - private final Optional streamName; + private final Optional streamIdentifier; private final String shardId; private final String concurrencyToken; // Sorted list of parent shardIds. @@ -67,7 +67,7 @@ public class ShardInfo { final String concurrencyToken, final Collection parentShardIds, final ExtendedSequenceNumber checkpoint, - final String streamName) { + final String streamIdentifier) { this.shardId = shardId; this.concurrencyToken = concurrencyToken; this.parentShardIds = new LinkedList<>(); @@ -78,7 +78,7 @@ public class ShardInfo { // This makes it easy to check for equality in ShardInfo.equals method. Collections.sort(this.parentShardIds); this.checkpoint = checkpoint; - this.streamName = Optional.ofNullable(streamName); + this.streamIdentifier = Optional.ofNullable(streamIdentifier); } /** @@ -105,7 +105,7 @@ public class ShardInfo { @Override public int hashCode() { return new HashCodeBuilder() - .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamName.orElse("")).toHashCode(); + .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamIdentifier.orElse("")).toHashCode(); } /** @@ -130,7 +130,7 @@ public class ShardInfo { ShardInfo other = (ShardInfo) obj; return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken) .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId) - .append(streamName.orElse(""), other.streamName.orElse("")).isEquals(); + .append(streamIdentifier.orElse(""), other.streamIdentifier.orElse("")).isEquals(); } @@ -140,8 +140,8 @@ public class ShardInfo { * @return */ public static String getLeaseKey(ShardInfo shardInfo) { - return shardInfo.streamName().isPresent() ? - MultiStreamLease.getLeaseKey(shardInfo.streamName().get(), shardInfo.shardId()) : + return shardInfo.streamIdentifier().isPresent() ? + MultiStreamLease.getLeaseKey(shardInfo.streamIdentifier().get(), shardInfo.shardId()) : shardInfo.shardId(); } 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 c0d3913b..f8a11c0a 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 @@ -382,7 +382,7 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { public static ShardInfo convertLeaseToAssignment(final Lease lease) { if (lease instanceof MultiStreamLease) { return new ShardInfo(((MultiStreamLease) lease).shardId(), lease.concurrencyToken().toString(), lease.parentShardIds(), - lease.checkpoint(), ((MultiStreamLease) lease).streamName()); + lease.checkpoint(), ((MultiStreamLease) lease).streamIdentifier()); } else { return new ShardInfo(lease.leaseKey(), lease.concurrencyToken().toString(), lease.parentShardIds(), lease.checkpoint()); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java index 7525273f..b8637bdb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java @@ -21,7 +21,7 @@ public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer public Map toDynamoRecord(Lease lease) { final MultiStreamLease multiStreamLease = validateAndCast(lease); final Map result = super.toDynamoRecord(multiStreamLease); - result.put(STREAM_NAME_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamName())); + result.put(STREAM_NAME_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier())); result.put(SHARD_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.shardId())); return result; } @@ -30,7 +30,7 @@ public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer public MultiStreamLease fromDynamoRecord(Map dynamoRecord) { final MultiStreamLease multiStreamLease = (MultiStreamLease) super .fromDynamoRecord(dynamoRecord, new MultiStreamLease()); - multiStreamLease.streamName(DynamoUtils.safeGetString(dynamoRecord, STREAM_NAME_KEY)); + multiStreamLease.streamIdentifier(DynamoUtils.safeGetString(dynamoRecord, STREAM_NAME_KEY)); multiStreamLease.shardId(DynamoUtils.safeGetString(dynamoRecord, SHARD_ID_KEY)); return multiStreamLease; } @@ -40,7 +40,7 @@ public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer public Map getDynamoUpdateLeaseUpdate(Lease lease) { final MultiStreamLease multiStreamLease = validateAndCast(lease); final Map result = super.getDynamoUpdateLeaseUpdate(multiStreamLease); - result.put(STREAM_NAME_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamName()))); + result.put(STREAM_NAME_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier()))); result.put(SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.shardId()))); return result; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java index 4f1db733..03ddc6ee 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java @@ -21,8 +21,8 @@ import lombok.experimental.Accessors; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.LeaseCoordinator; -import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; @@ -41,7 +41,7 @@ public class ShardConsumerArgument { @NonNull private final ShardInfo shardInfo; @NonNull - private final String streamName; + private final StreamIdentifier streamIdentifier; @NonNull private final LeaseCoordinator leaseCoordinator; @NonNull 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 fae7e4cb..e41f7e08 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 @@ -2,6 +2,7 @@ package software.amazon.kinesis.processor; import software.amazon.kinesis.common.StreamConfig; +import java.util.List; import java.util.Map; /** @@ -15,5 +16,5 @@ public interface MultiStreamTracker { * * @return List of stream names */ - Map streamConfigMap(); + List streamConfigList(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java index 4b691401..e16695b2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java @@ -15,6 +15,8 @@ package software.amazon.kinesis.processor; +import software.amazon.kinesis.common.StreamIdentifier; + /** * */ @@ -28,10 +30,10 @@ public interface ShardRecordProcessorFactory { /** * Returns a new instance of the ShardRecordProcessor for a stream - * @param streamName + * @param streamIdentifier * @return ShardRecordProcessor */ - default ShardRecordProcessor shardRecordProcessor(String streamName) { + default ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { return shardRecordProcessor(); } } 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 5c0f5e9a..d4654323 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 @@ -47,7 +47,7 @@ public class FanOutRetrievalFactory implements RetrievalFactory { @Override public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, final MetricsFactory metricsFactory) { - final String streamName = shardInfo.streamName().orElse(defaultStreamName); + final String streamName = shardInfo.streamIdentifier().orElse(defaultStreamName); return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), streamToconsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java index 7405730e..4a0dc9d5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java @@ -63,7 +63,7 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory { public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, shardInfo.streamName().orElse(streamName), shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); + new KinesisDataFetcher(kinesisClient, shardInfo.streamIdentifier().orElse(streamName), shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java index 8b669893..6e2e2b05 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java @@ -68,7 +68,7 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, shardInfo.streamName().orElse(streamName), shardInfo.shardId(), + new KinesisDataFetcher(kinesisClient, shardInfo.streamIdentifier().orElse(streamName), shardInfo.shardId(), maxRecords, metricsFactory, maxFutureWait)); } From 8e8f6ed3527a3ea333b4564c8835e068e5d4fad6 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 4 Mar 2020 02:52:20 -0800 Subject: [PATCH 014/159] Adding account and stream epoch support. Checkpoint 2 --- .../kinesis/common/StreamIdentifier.java | 2 ++ .../leases/HierarchicalShardSyncer.java | 23 +++++++++--------- .../kinesis/leases/KinesisShardDetector.java | 15 ++++++------ .../kinesis/leases/MultiStreamLease.java | 6 ++--- .../amazon/kinesis/leases/ShardDetector.java | 3 ++- .../DynamoDBLeaseManagementFactory.java | 13 +++++++--- .../DynamoDBMultiStreamLeaseSerializer.java | 9 +++---- .../fanout/FanOutRetrievalFactory.java | 14 ++++++++--- .../retrieval/polling/KinesisDataFetcher.java | 12 ++++++---- .../SynchronousBlockingRetrievalFactory.java | 6 ++++- ...ynchronousPrefetchingRetrievalFactory.java | 6 ++++- .../kinesis/coordinator/SchedulerTest.java | 4 +++- .../kinesis/lifecycle/ConsumerStatesTest.java | 3 ++- .../retrieval/fanout/FanOutConfigTest.java | 24 +++++++++++++++---- 14 files changed, 94 insertions(+), 46 deletions(-) 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 e54c97e6..5b1f9977 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 @@ -3,11 +3,13 @@ package software.amazon.kinesis.common; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.RequiredArgsConstructor; +import lombok.experimental.Accessors; import software.amazon.awssdk.utils.Validate; @RequiredArgsConstructor @EqualsAndHashCode @Getter +@Accessors(fluent = true) public class StreamIdentifier { private final String accountName; private final String streamName; 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 459d730d..1102574f 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 @@ -41,6 +41,7 @@ import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; @@ -120,9 +121,9 @@ public class HierarchicalShardSyncer { assertAllParentShardsAreClosed(inconsistentShardIds); } final List currentLeases = isMultiStreamMode ? - getLeasesForStream(shardDetector.streamName(), leaseRefresher) : + getLeasesForStream(shardDetector.streamIdentifier(), leaseRefresher) : leaseRefresher.listLeases(); - final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamName()); + final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamIdentifier()); final List newLeasesToCreate = determineNewLeasesToCreate(latestShards, currentLeases, initialPosition, inconsistentShardIds, multiStreamArgs); log.debug("Num new leases to create: {}", newLeasesToCreate.size()); @@ -149,19 +150,19 @@ public class HierarchicalShardSyncer { /** Note: This method has package level access solely for testing purposes. * - * @param streamName We'll use this stream name to filter leases + * @param streamIdentifier We'll use this stream identifier to filter leases * @param leaseRefresher Used to fetch leases * @return Return list of leases (corresponding to shards) of the specified stream. * @throws DependencyException * @throws InvalidStateException * @throws ProvisionedThroughputException */ - static List getLeasesForStream(String streamName, + static List getLeasesForStream(StreamIdentifier streamIdentifier, LeaseRefresher leaseRefresher) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamLeases = new ArrayList<>(); for (Lease lease : leaseRefresher.listLeases()) { - if (streamName.equals(((MultiStreamLease)lease).streamIdentifier())) { + if (streamIdentifier.toString().equals(((MultiStreamLease)lease).streamIdentifier())) { streamLeases.add(lease); } } @@ -378,7 +379,7 @@ public class HierarchicalShardSyncer { } else { log.debug("Need to create a lease for shardId {}", shardId); final Lease newLease = multiStreamArgs.isMultiStreamMode() ? - newKCLMultiStreamLease(shard, multiStreamArgs.streamName()) : + newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : newKCLLease(shard); final boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, @@ -502,7 +503,7 @@ public class HierarchicalShardSyncer { if (lease == null) { lease = multiStreamArgs.isMultiStreamMode() ? newKCLMultiStreamLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId), - multiStreamArgs.streamName()) : + multiStreamArgs.streamIdentifier()) : newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); shardIdToLeaseMapOfNewShards.put(parentShardId, lease); } @@ -758,9 +759,9 @@ public class HierarchicalShardSyncer { return newLease; } - private static Lease newKCLMultiStreamLease(final Shard shard, final String streamName) { + private static Lease newKCLMultiStreamLease(final Shard shard, final StreamIdentifier streamIdentifier) { MultiStreamLease newLease = new MultiStreamLease(); - newLease.leaseKey(MultiStreamLease.getLeaseKey(streamName, shard.shardId())); + newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.toString(), shard.shardId())); List parentShardIds = new ArrayList<>(2); if (shard.parentShardId() != null) { parentShardIds.add(shard.parentShardId()); @@ -770,7 +771,7 @@ public class HierarchicalShardSyncer { } newLease.parentShardIds(parentShardIds); newLease.ownerSwitchesSinceCheckpoint(0L); - newLease.streamIdentifier(streamName); + newLease.streamIdentifier(streamIdentifier.toString()); newLease.shardId(shard.shardId()); return newLease; } @@ -858,7 +859,7 @@ public class HierarchicalShardSyncer { @Accessors(fluent = true) private static class MultiStreamArgs { private final Boolean isMultiStreamMode; - private final String streamName; + private final StreamIdentifier streamIdentifier; } } 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 f009a0e7..53751375 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 @@ -47,6 +47,7 @@ import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; import software.amazon.kinesis.common.KinesisRequestsBuilder; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.retrieval.AWSExceptionManager; /** @@ -60,7 +61,7 @@ public class KinesisShardDetector implements ShardDetector { @NonNull private final KinesisAsyncClient kinesisClient; @NonNull @Getter - private final String streamName; + private final StreamIdentifier streamIdentifier; private final long listShardsBackoffTimeInMillis; private final int maxListShardsRetryAttempts; private final long listShardsCacheAllowedAgeInSeconds; @@ -77,16 +78,16 @@ public class KinesisShardDetector implements ShardDetector { public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload, int cacheMissWarningModulus) { - this(kinesisClient, streamName, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, + this(kinesisClient, StreamIdentifier.fromStreamName(streamName), listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); } - public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, + public KinesisShardDetector(KinesisAsyncClient kinesisClient, StreamIdentifier streamIdentifier, long listShardsBackoffTimeInMillis, int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload, int cacheMissWarningModulus, Duration kinesisRequestTimeout) { this.kinesisClient = kinesisClient; - this.streamName = streamName; + this.streamIdentifier = streamIdentifier; this.listShardsBackoffTimeInMillis = listShardsBackoffTimeInMillis; this.maxListShardsRetryAttempts = maxListShardsRetryAttempts; this.listShardsCacheAllowedAgeInSeconds = listShardsCacheAllowedAgeInSeconds; @@ -180,7 +181,7 @@ public class KinesisShardDetector implements ShardDetector { ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder(); if (StringUtils.isEmpty(nextToken)) { - request = request.streamName(streamName); + request = request.streamName(streamIdentifier.streamName()); } else { request = request.nextToken(nextToken); } @@ -205,12 +206,12 @@ public class KinesisShardDetector implements ShardDetector { + " Active or Updating)"); return null; } catch (LimitExceededException e) { - log.info("Got LimitExceededException when listing shards {}. Backing off for {} millis.", streamName, + log.info("Got LimitExceededException when listing shards {}. Backing off for {} millis.", streamIdentifier, listShardsBackoffTimeInMillis); try { Thread.sleep(listShardsBackoffTimeInMillis); } catch (InterruptedException ie) { - log.debug("Stream {} : Sleep was interrupted ", streamName, ie); + log.debug("Stream {} : Sleep was interrupted ", streamIdentifier, ie); } lastException = e; } catch (TimeoutException te) { 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 8b29168d..aa850f2f 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 @@ -35,10 +35,10 @@ public class MultiStreamLease extends Lease { shardId(casted.shardId); } - public static String getLeaseKey(String streamName, String shardId) { - verifyNotNull(streamName, "streamName should not be null"); + public static String getLeaseKey(String streamIdentifier, String shardId) { + verifyNotNull(streamIdentifier, "streamIdentifier should not be null"); verifyNotNull(shardId, "shardId should not be null"); - return streamName + ":" + shardId; + return streamIdentifier + ":" + shardId; } @Override 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 43e1e1b3..6ae012e6 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 @@ -16,6 +16,7 @@ package software.amazon.kinesis.leases; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.kinesis.common.StreamIdentifier; import java.util.List; @@ -27,7 +28,7 @@ public interface ShardDetector { List listShards(); - default String streamName() { + default StreamIdentifier streamIdentifier() { throw new UnsupportedOperationException("StreamName not available"); } 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 124d28ca..e383a41b 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 @@ -31,6 +31,7 @@ import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.KinesisShardDetector; import software.amazon.kinesis.leases.LeaseCoordinator; @@ -330,7 +331,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode) { - this(kinesisClient, new StreamConfig(streamName, initialPositionInStream), dynamoDBClient, tableName, + this(kinesisClient, new StreamConfig(StreamIdentifier.fromStreamName(streamName), initialPositionInStream), dynamoDBClient, tableName, workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, @@ -477,14 +478,20 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @Override @Deprecated public ShardDetector createShardDetector() { - return new KinesisShardDetector(kinesisClient, streamConfig.streamName(), + return new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout); } + /** + * KinesisShardDetector supports reading from service only using streamName. Support for accountId and + * stream creation epoch is yet to be provided. + * @param streamConfig + * @return + */ @Override public ShardDetector createShardDetector(StreamConfig streamConfig) { - return new KinesisShardDetector(kinesisClient, streamConfig.streamName(), listShardsBackoffTimeMillis, + return new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java index b8637bdb..d703a970 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java @@ -14,14 +14,15 @@ import static software.amazon.kinesis.leases.MultiStreamLease.validateAndCast; @NoArgsConstructor public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer { - private static final String STREAM_NAME_KEY = "streamName"; + // Keeping the stream id as "streamName" for legacy reasons. + private static final String STREAM_ID_KEY = "streamName"; private static final String SHARD_ID_KEY = "shardId"; @Override public Map toDynamoRecord(Lease lease) { final MultiStreamLease multiStreamLease = validateAndCast(lease); final Map result = super.toDynamoRecord(multiStreamLease); - result.put(STREAM_NAME_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier())); + result.put(STREAM_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier())); result.put(SHARD_ID_KEY, DynamoUtils.createAttributeValue(multiStreamLease.shardId())); return result; } @@ -30,7 +31,7 @@ public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer public MultiStreamLease fromDynamoRecord(Map dynamoRecord) { final MultiStreamLease multiStreamLease = (MultiStreamLease) super .fromDynamoRecord(dynamoRecord, new MultiStreamLease()); - multiStreamLease.streamIdentifier(DynamoUtils.safeGetString(dynamoRecord, STREAM_NAME_KEY)); + multiStreamLease.streamIdentifier(DynamoUtils.safeGetString(dynamoRecord, STREAM_ID_KEY)); multiStreamLease.shardId(DynamoUtils.safeGetString(dynamoRecord, SHARD_ID_KEY)); return multiStreamLease; } @@ -40,7 +41,7 @@ public class DynamoDBMultiStreamLeaseSerializer extends DynamoDBLeaseSerializer public Map getDynamoUpdateLeaseUpdate(Lease lease) { final MultiStreamLease multiStreamLease = validateAndCast(lease); final Map result = super.getDynamoUpdateLeaseUpdate(multiStreamLease); - result.put(STREAM_NAME_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier()))); + result.put(STREAM_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.streamIdentifier()))); result.put(SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(multiStreamLease.shardId()))); return result; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java index d4654323..f655d8fd 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java @@ -19,6 +19,7 @@ import lombok.NonNull; import lombok.RequiredArgsConstructor; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @@ -27,6 +28,7 @@ import software.amazon.kinesis.retrieval.RetrievalFactory; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.function.Function; @RequiredArgsConstructor @@ -36,7 +38,7 @@ public class FanOutRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; private final String defaultStreamName; private final Function consumerArnProvider; - private Map streamToconsumerArnMap = new HashMap<>(); + private Map streamToConsumerArnMap = new HashMap<>(); @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo, @@ -47,8 +49,14 @@ public class FanOutRetrievalFactory implements RetrievalFactory { @Override public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, final MetricsFactory metricsFactory) { - final String streamName = shardInfo.streamIdentifier().orElse(defaultStreamName); + final Optional streamIdentifierStr = shardInfo.streamIdentifier(); + final String streamName; + if(streamIdentifierStr.isPresent()) { + streamName = StreamIdentifier.fromString(streamIdentifierStr.get()).streamName(); + } else { + streamName = defaultStreamName; + } return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), - streamToconsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); + streamToConsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); } } 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 1b769cb9..db6d2aae 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 @@ -40,6 +40,7 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.KinesisRequestsBuilder; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsLevel; import software.amazon.kinesis.metrics.MetricsScope; @@ -63,7 +64,7 @@ public class KinesisDataFetcher { @NonNull private final KinesisAsyncClient kinesisClient; @NonNull - private final String streamName; + private final StreamIdentifier streamIdentifier; @NonNull private final String shardId; private final int maxRecords; @@ -73,12 +74,13 @@ public class KinesisDataFetcher { @Deprecated public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory) { - this(kinesisClient, streamName, shardId, maxRecords, metricsFactory, PollingConfig.DEFAULT_REQUEST_TIMEOUT); + this(kinesisClient, StreamIdentifier.fromStreamName(streamName), shardId, maxRecords, metricsFactory, PollingConfig.DEFAULT_REQUEST_TIMEOUT); } - public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory, Duration maxFutureWait) { + // Changing the constructor directly as this is an internal API + public KinesisDataFetcher(KinesisAsyncClient kinesisClient, StreamIdentifier streamIdentifier, String shardId, int maxRecords, MetricsFactory metricsFactory, Duration maxFutureWait) { this.kinesisClient = kinesisClient; - this.streamName = streamName; + this.streamIdentifier = streamIdentifier; this.shardId = shardId; this.maxRecords = maxRecords; this.metricsFactory = metricsFactory; @@ -199,7 +201,7 @@ public class KinesisDataFetcher { final AWSExceptionManager exceptionManager = createExceptionManager(); GetShardIteratorRequest.Builder builder = KinesisRequestsBuilder.getShardIteratorRequestBuilder() - .streamName(streamName).shardId(shardId); + .streamName(streamIdentifier.streamName()).shardId(shardId); GetShardIteratorRequest request = IteratorBuilder.request(builder, sequenceNumber, initialPositionInStream) .build(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java index 4a0dc9d5..93718f7c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java @@ -19,6 +19,7 @@ import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @@ -62,8 +63,11 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory { @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { + final StreamIdentifier streamIdentifier = shardInfo.streamIdentifier().isPresent() ? + StreamIdentifier.fromString(shardInfo.streamIdentifier().get()) : + StreamIdentifier.fromStreamName(streamName); return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, shardInfo.streamIdentifier().orElse(streamName), shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); + new KinesisDataFetcher(kinesisClient, streamIdentifier, shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java index 6e2e2b05..e73ad928 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java @@ -21,6 +21,7 @@ import java.util.concurrent.ExecutorService; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @@ -67,8 +68,11 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { + final StreamIdentifier streamIdentifier = shardInfo.streamIdentifier().isPresent() ? + StreamIdentifier.fromString(shardInfo.streamIdentifier().get()) : + StreamIdentifier.fromStreamName(streamName); return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, shardInfo.streamIdentifier().orElse(streamName), shardInfo.shardId(), + new KinesisDataFetcher(kinesisClient, streamIdentifier, shardInfo.shardId(), maxRecords, metricsFactory, maxFutureWait)); } 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 a17ad4cb..7cfc86a5 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 @@ -53,6 +53,7 @@ import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -501,9 +502,10 @@ public class SchedulerTest { } @Override - public ShardRecordProcessor shardRecordProcessor(String streamName) { + public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { return shardRecordProcessor(); } + } private class TestKinesisLeaseManagementFactory implements LeaseManagementFactory { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java index 16f5e9a4..810f0159 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 @@ -43,6 +43,7 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; @@ -114,7 +115,7 @@ public class ConsumerStatesTest { @Before public void setup() { - argument = new ShardConsumerArgument(shardInfo, STREAM_NAME, leaseCoordinator, executorService, recordsPublisher, + argument = new ShardConsumerArgument(shardInfo, StreamIdentifier.fromStreamName(STREAM_NAME), leaseCoordinator, executorService, recordsPublisher, shardRecordProcessor, checkpointer, recordProcessorCheckpointer, parentShardPollIntervalMillis, taskBackoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, shouldCallProcessRecordsEvenForEmptyRecordList, idleTimeInMillis, diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java index a0d18d56..cd3e8af8 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java @@ -19,6 +19,8 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.nullValue; import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -29,9 +31,14 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.RetrievalFactory; +import java.util.Optional; + @RunWith(MockitoJUnitRunner.class) public class FanOutConfigTest { @@ -59,7 +66,9 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) .streamName(TEST_STREAM_NAME); RetrievalFactory retrievalFactory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + retrievalFactory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(retrievalFactory, not(nullValue())); verify(consumerRegistration).getOrCreateStreamConsumerArn(); } @@ -83,7 +92,9 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) .streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; @@ -96,9 +107,10 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).consumerName(TEST_CONSUMER_NAME) .streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); - TestingConfig testingConfig = (TestingConfig) config; assertThat(testingConfig.stream, equalTo(TEST_STREAM_NAME)); assertThat(testingConfig.consumerToCreate, equalTo(TEST_CONSUMER_NAME)); @@ -109,7 +121,9 @@ public class FanOutConfigTest { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) .consumerName(TEST_CONSUMER_NAME).streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); - + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; From 255ae932d2a0719d543f9f615ffc030b181d61d2 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 12 Mar 2020 14:15:36 -0700 Subject: [PATCH 015/159] Adding account and stream epoch support. Checkpoint 3 --- .../amazon/kinesis/common/ConfigsBuilder.java | 66 +++++++++++++++---- .../amazon/kinesis/common/StreamConfig.java | 7 +- .../kinesis/common/StreamIdentifier.java | 22 +++++-- .../amazon/kinesis/coordinator/Scheduler.java | 38 +++++------ .../kinesis/leases/MultiStreamLease.java | 16 ++--- .../kinesis/retrieval/RetrievalConfig.java | 50 ++++++++++---- 6 files changed, 130 insertions(+), 69 deletions(-) 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 7026e34c..9595fdf9 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java @@ -15,14 +15,18 @@ package software.amazon.kinesis.common; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.Setter; +import lombok.ToString; import org.apache.commons.lang3.StringUtils; -import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.coordinator.CoordinatorConfig; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -36,14 +40,15 @@ import software.amazon.kinesis.retrieval.RetrievalConfig; /** * This Builder is useful to create all configurations for the KCL with default values. */ -@Data +@Getter @Setter @ToString @EqualsAndHashCode @Accessors(fluent = true) public class ConfigsBuilder { /** - * Name of the stream to consume records from + * Either the name of the stream to consume records from + * Or MultiStreamTracker for all the streams to consume records from */ - @NonNull - private final String streamName; + private Either appStreamTracker; + /** * Application name for the KCL Worker */ @@ -109,7 +114,45 @@ public class ConfigsBuilder { return namespace; } - private MultiStreamTracker multiStreamTracker; + /** + * Constructor to initialize ConfigsBuilder with StreamName + * @param streamName + * @param applicationName + * @param kinesisClient + * @param dynamoDBClient + * @param cloudWatchClient + * @param workerIdentifier + * @param shardRecordProcessorFactory + */ + public ConfigsBuilder(@NonNull String streamName, @NonNull String applicationName, @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + this.appStreamTracker = Either.right(streamName); + this.applicationName = applicationName; + this.kinesisClient = kinesisClient; + this.dynamoDBClient = dynamoDBClient; + this.cloudWatchClient = cloudWatchClient; + this.workerIdentifier = workerIdentifier; + this.shardRecordProcessorFactory = shardRecordProcessorFactory; + } + + /** + * Constructor to initialize ConfigsBuilder with MultiStreamTracker + * @param multiStreamTracker + * @param applicationName + * @param kinesisClient + * @param dynamoDBClient + * @param cloudWatchClient + * @param workerIdentifier + * @param shardRecordProcessorFactory + */ + public ConfigsBuilder(@NonNull MultiStreamTracker multiStreamTracker, @NonNull String applicationName, @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + this.appStreamTracker = Either.left(multiStreamTracker); + this.applicationName = applicationName; + this.kinesisClient = kinesisClient; + this.dynamoDBClient = dynamoDBClient; + this.cloudWatchClient = cloudWatchClient; + this.workerIdentifier = workerIdentifier; + this.shardRecordProcessorFactory = shardRecordProcessorFactory; + } /** * Creates a new instance of CheckpointConfig @@ -135,8 +178,7 @@ public class ConfigsBuilder { * @return LeaseManagementConfig */ public LeaseManagementConfig leaseManagementConfig() { - return new LeaseManagementConfig(tableName(), dynamoDBClient(), kinesisClient(), streamName(), - workerIdentifier()); + return new LeaseManagementConfig(tableName(), dynamoDBClient(), kinesisClient(), workerIdentifier()); } /** @@ -173,10 +215,10 @@ public class ConfigsBuilder { * @return RetrievalConfig */ public RetrievalConfig retrievalConfig() { - final RetrievalConfig retrievalConfig = new RetrievalConfig(kinesisClient(), streamName(), applicationName()); - if(this.multiStreamTracker != null) { - retrievalConfig.multiStreamTracker(multiStreamTracker); - } + final RetrievalConfig retrievalConfig = + appStreamTracker.map( + multiStreamTracker -> new RetrievalConfig(kinesisClient(), multiStreamTracker, applicationName()), + streamName -> new RetrievalConfig(kinesisClient(), streamName, applicationName())); return retrievalConfig; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java index 667f1f1c..999182b6 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 @@ -1,13 +1,10 @@ package software.amazon.kinesis.common; -import lombok.AccessLevel; -import lombok.Data; +import lombok.Value; import lombok.experimental.Accessors; -import lombok.experimental.FieldDefaults; -@Data +@Value @Accessors(fluent = true) -@FieldDefaults(makeFinal=true, level= AccessLevel.PRIVATE) public class StreamConfig { StreamIdentifier streamIdentifier; InitialPositionInStreamExtended initialPositionInStreamExtended; 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 5b1f9977..2435b5bf 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 @@ -1,11 +1,14 @@ package software.amazon.kinesis.common; +import com.google.common.base.Joiner; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.RequiredArgsConstructor; import lombok.experimental.Accessors; import software.amazon.awssdk.utils.Validate; +import java.util.regex.Pattern; + @RequiredArgsConstructor @EqualsAndHashCode @Getter @@ -13,23 +16,28 @@ import software.amazon.awssdk.utils.Validate; public class StreamIdentifier { private final String accountName; private final String streamName; - private final String streamCreationEpoch; + private final Long streamCreationEpoch; - private static final String DEFAULT = "default"; + private static final String DEFAULT_ACCOUNT = "default"; + private static final String DELIMITER = ":"; + private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*"); @Override public String toString(){ - return accountName + ":" + streamName + ":" + streamCreationEpoch; + return Joiner.on(DELIMITER).join(accountName, streamName, streamCreationEpoch); } public static StreamIdentifier fromString(String streamIdentifier) { - final String[] idTokens = streamIdentifier.split(":"); - Validate.isTrue(idTokens.length == 3, "Unable to deserialize StreamIdentifier from " + streamIdentifier); - return new StreamIdentifier(idTokens[0], idTokens[1], idTokens[2]); + if (PATTERN.matcher(streamIdentifier).matches()) { + final String[] split = streamIdentifier.split(DELIMITER); + return new StreamIdentifier(split[0], split[1], Long.parseLong(split[2])); + } else { + throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifier); + } } public static StreamIdentifier fromStreamName(String streamName) { Validate.notEmpty(streamName, "StreamName should not be empty"); - return new StreamIdentifier(DEFAULT, streamName, DEFAULT); + return new StreamIdentifier(DEFAULT_ACCOUNT, streamName, 0L); } } 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 c94981b9..1e95f2e6 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 @@ -16,6 +16,7 @@ package software.amazon.kinesis.coordinator; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -77,7 +78,6 @@ import software.amazon.kinesis.processor.Checkpointer; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShutdownNotificationAware; -import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; @@ -122,7 +122,7 @@ public class Scheduler implements Runnable { private final MetricsFactory metricsFactory; private final long failoverTimeMillis; private final long taskBackoffTimeMillis; - private final Either appStreamTracker; + private final boolean isMultiStreamMode; private final Map currentStreamConfigMap; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; @@ -183,27 +183,22 @@ public class Scheduler implements Runnable { this.retrievalConfig = retrievalConfig; this.applicationName = this.coordinatorConfig.applicationName(); - final MultiStreamTracker multiStreamTracker = this.retrievalConfig.multiStreamTracker(); - if(multiStreamTracker == null) { - final StreamConfig streamConfig = new StreamConfig(StreamIdentifier.fromStreamName(this.retrievalConfig.streamName()), - this.retrievalConfig.initialPositionInStreamExtended()); - this.appStreamTracker = Either.right(streamConfig); - this.currentStreamConfigMap = new HashMap() {{ - put(streamConfig.streamIdentifier(), streamConfig); - }}; - } else { - this.appStreamTracker = Either.left(multiStreamTracker); - this.currentStreamConfigMap = multiStreamTracker.streamConfigList().stream() - .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); - } + this.isMultiStreamMode = this.retrievalConfig.appStreamTracker().map( + multiStreamTracker -> true, streamConfig -> false); + this.currentStreamConfigMap = this.retrievalConfig.appStreamTracker().map( + multiStreamTracker -> + multiStreamTracker.streamConfigList().stream() + .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)), + streamConfig -> + Collections.singletonMap(streamConfig.streamIdentifier(), streamConfig)); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); // Determine leaseSerializer based on availability of MultiStreamTracker. - final LeaseSerializer leaseSerializer = this.appStreamTracker.map(mst -> true, sc -> false) ? + final LeaseSerializer leaseSerializer = isMultiStreamMode ? new DynamoDBMultiStreamLeaseSerializer() : new DynamoDBLeaseSerializer(); this.leaseCoordinator = this.leaseManagementConfig - .leaseManagementFactory(leaseSerializer, this.appStreamTracker.map(mst -> true, sc -> false)) + .leaseManagementFactory(leaseSerializer, isMultiStreamMode) .createLeaseCoordinator(this.metricsFactory); this.leaseRefresher = this.leaseCoordinator.leaseRefresher(); @@ -224,7 +219,7 @@ public class Scheduler implements Runnable { // TODO : Halo : Handle case of no StreamConfig present in streamConfigList() for the supplied streamName. // TODO : Pass the immutable map here instead of using mst.streamConfigList() this.shardSyncTaskManagerProvider = streamIdentifier -> this.leaseManagementConfig - .leaseManagementFactory(leaseSerializer, this.appStreamTracker.map(mst -> true, sc -> false)) + .leaseManagementFactory(leaseSerializer, isMultiStreamMode) .createShardSyncTaskManager(this.metricsFactory, this.currentStreamConfigMap.get(streamIdentifier)); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); @@ -252,8 +247,7 @@ public class Scheduler implements Runnable { this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); // TODO : Halo : Check if this needs to be per stream. - this.hierarchicalShardSyncer = leaseManagementConfig - .hierarchicalShardSyncer(this.appStreamTracker.map(mst -> true, sc -> false)); + this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(isMultiStreamMode); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); } @@ -643,7 +637,6 @@ public class Scheduler implements Runnable { // get the default stream name for the single stream application. final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifier()); // Irrespective of single stream app or multi stream app, streamConfig should always be available. - // TODO: Halo : if not available, construct a default config ? final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); Validate.notNull(streamConfig, "StreamConfig should not be empty"); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, @@ -721,7 +714,8 @@ public class Scheduler implements Runnable { if(streamIdentifierString.isPresent()) { streamIdentifier = StreamIdentifier.fromString(streamIdentifierString.get()); } else { - streamIdentifier = appStreamTracker.map(mst -> null, sc -> sc.streamIdentifier()); + Validate.isTrue(!isMultiStreamMode, "Should not be in MultiStream Mode"); + streamIdentifier = this.currentStreamConfigMap.values().iterator().next().streamIdentifier(); } Validate.notNull(streamIdentifier, "Stream identifier should not be empty"); return streamIdentifier; 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 aa850f2f..862c203d 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 @@ -20,11 +20,10 @@ public class MultiStreamLease extends Lease { @NonNull private String streamIdentifier; @NonNull private String shardId; - public MultiStreamLease(Lease other) { + public MultiStreamLease(MultiStreamLease other) { super(other); - MultiStreamLease casted = validateAndCast(other); - streamIdentifier(casted.streamIdentifier); - shardId(casted.shardId); + streamIdentifier(other.streamIdentifier); + shardId(other.shardId); } @Override @@ -58,14 +57,7 @@ public class MultiStreamLease extends Lease { return false; } MultiStreamLease other = (MultiStreamLease) obj; - if (streamIdentifier == null) { - if (other.streamIdentifier != null) { - return false; - } - } else if (!streamIdentifier.equals(other.streamIdentifier)) { - return false; - } - return true; + return Objects.equals(streamIdentifier, other.streamIdentifier); } /** 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 10b23641..f26c0e13 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java @@ -15,19 +15,25 @@ package software.amazon.kinesis.retrieval; -import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.Getter; import lombok.NonNull; +import lombok.Setter; +import lombok.ToString; import lombok.experimental.Accessors; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.fanout.FanOutConfig; /** * Used by the KCL to configure the retrieval of records from Kinesis. */ -@Data +@Getter @Setter @ToString @EqualsAndHashCode @Accessors(fluent = true) public class RetrievalConfig { /** @@ -43,19 +49,13 @@ public class RetrievalConfig { @NonNull private final KinesisAsyncClient kinesisClient; - /** - * The name of the stream to process records from. - */ - @NonNull - private final String streamName; - @NonNull private final String applicationName; /** - * StreamTracker for multi streaming support + * AppStreamTracker either for multi stream tracking or single stream */ - private MultiStreamTracker multiStreamTracker; + private Either appStreamTracker; /** * Backoff time between consecutive ListShards calls. @@ -90,15 +90,43 @@ public class RetrievalConfig { private RetrievalFactory retrievalFactory; + public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName, + @NonNull String applicationName) { + this.kinesisClient = kinesisAsyncClient; + this.appStreamTracker = Either + .right(new StreamConfig(StreamIdentifier.fromStreamName(streamName), initialPositionInStreamExtended)); + this.applicationName = applicationName; + } + + public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull MultiStreamTracker multiStreamTracker, + @NonNull String applicationName) { + this.kinesisClient = kinesisAsyncClient; + this.appStreamTracker = Either.left(multiStreamTracker); + this.applicationName = applicationName; + } + + public void 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)); + } + public RetrievalFactory retrievalFactory() { if (retrievalFactory == null) { if (retrievalSpecificConfig == null) { - retrievalSpecificConfig = new FanOutConfig(kinesisClient()).streamName(streamName()) + retrievalSpecificConfig = new FanOutConfig(kinesisClient()) .applicationName(applicationName()); + retrievalSpecificConfig = appStreamTracker.map(multiStreamTracker -> retrievalSpecificConfig, + streamConfig -> ((FanOutConfig)retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName())); } retrievalFactory = retrievalSpecificConfig.retrievalFactory(); } return retrievalFactory; } + } From 2e113dbd6ca0573d80d01579c33d5c71ba876816 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 13 Mar 2020 01:36:33 -0700 Subject: [PATCH 016/159] Added Unit Test cases, code comments and other code refactoring --- .../kinesis/common/StreamIdentifier.java | 62 ++-- .../amazon/kinesis/coordinator/Scheduler.java | 14 +- .../leases/HierarchicalShardSyncer.java | 10 +- .../kinesis/leases/KinesisShardDetector.java | 3 +- .../kinesis/leases/LeaseManagementConfig.java | 25 +- .../amazon/kinesis/leases/ShardInfo.java | 27 +- .../dynamodb/DynamoDBLeaseCoordinator.java | 6 +- .../DynamoDBLeaseManagementFactory.java | 47 ++- .../kinesis/processor/MultiStreamTracker.java | 4 +- .../ShardRecordProcessorFactory.java | 2 +- .../kinesis/retrieval/RetrievalConfig.java | 2 +- .../retrieval/fanout/FanOutConfig.java | 1 + .../fanout/FanOutRetrievalFactory.java | 4 +- .../retrieval/polling/KinesisDataFetcher.java | 12 +- .../SynchronousBlockingRetrievalFactory.java | 6 +- ...ynchronousPrefetchingRetrievalFactory.java | 6 +- .../kinesis/coordinator/SchedulerTest.java | 130 ++++++++- .../leases/HierarchicalShardSyncerTest.java | 269 ++++++++++++++++++ .../kinesis/lifecycle/ConsumerStatesTest.java | 4 +- .../retrieval/fanout/FanOutConfigTest.java | 10 +- 20 files changed, 562 insertions(+), 82 deletions(-) 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 2435b5bf..0bdce7fb 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 @@ -3,41 +3,67 @@ package software.amazon.kinesis.common; import com.google.common.base.Joiner; import lombok.EqualsAndHashCode; import lombok.Getter; -import lombok.RequiredArgsConstructor; import lombok.experimental.Accessors; import software.amazon.awssdk.utils.Validate; +import java.util.Optional; import java.util.regex.Pattern; -@RequiredArgsConstructor -@EqualsAndHashCode -@Getter -@Accessors(fluent = true) +@EqualsAndHashCode @Getter @Accessors(fluent = true) public class StreamIdentifier { - private final String accountName; + private final Optional accountIdOptional; private final String streamName; - private final Long streamCreationEpoch; + private final Optional streamCreationEpochOptional; - private static final String DEFAULT_ACCOUNT = "default"; private static final String DELIMITER = ":"; private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*"); - @Override - public String toString(){ - return Joiner.on(DELIMITER).join(accountName, streamName, streamCreationEpoch); + private StreamIdentifier(Optional accountIdOptional, String streamName, + Optional streamCreationEpochOptional) { + Validate.isTrue((accountIdOptional.isPresent() && streamCreationEpochOptional.isPresent()) || + (!accountIdOptional.isPresent() && !streamCreationEpochOptional.isPresent()), + "AccountId and StreamCreationEpoch must either be present together or not"); + this.accountIdOptional = accountIdOptional; + this.streamName = streamName; + this.streamCreationEpochOptional = streamCreationEpochOptional; } - public static StreamIdentifier fromString(String streamIdentifier) { - if (PATTERN.matcher(streamIdentifier).matches()) { - final String[] split = streamIdentifier.split(DELIMITER); - return new StreamIdentifier(split[0], split[1], Long.parseLong(split[2])); + /** + * Serialize the current StreamIdentifier instance. + * @return + */ + public String serialize() { + return accountIdOptional.isPresent() ? + Joiner.on(DELIMITER).join(accountIdOptional.get(), streamName, streamCreationEpochOptional.get()) : + streamName; + } + + @Override + public String toString() { + return serialize(); + } + + /** + * Create a multi stream instance for StreamIdentifier from serialized stream identifier. + * @param streamIdentifierSer + * @return StreamIdentifier + */ + public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) { + if (PATTERN.matcher(streamIdentifierSer).matches()) { + final String[] split = streamIdentifierSer.split(DELIMITER); + return new StreamIdentifier(Optional.of(split[0]), split[1], Optional.of(Long.parseLong(split[2]))); } else { - throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifier); + throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer); } } - public static StreamIdentifier fromStreamName(String streamName) { + /** + * Create a single stream instance for StreamIdentifier from stream name. + * @param streamName + * @return StreamIdentifier + */ + public static StreamIdentifier singleStreamInstance(String streamName) { Validate.notEmpty(streamName, "StreamName should not be empty"); - return new StreamIdentifier(DEFAULT_ACCOUNT, streamName, 0L); + return new StreamIdentifier(Optional.empty(), streamName, Optional.empty()); } } 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 1e95f2e6..48467dfb 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 @@ -42,7 +42,6 @@ import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; -import software.amazon.awssdk.utils.Either; import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; @@ -295,9 +294,10 @@ public class Scheduler implements Runnable { if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - for(StreamIdentifier streamIdentifier : currentStreamConfigMap.keySet().stream().collect(Collectors.toList())) { - log.info("Syncing Kinesis shard info"); - final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); + for(Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); + log.info("Syncing Kinesis shard info for " + streamIdentifier); + final StreamConfig streamConfig = streamConfigEntry.getValue(); ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamIdentifier), leaseRefresher, streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, @@ -363,7 +363,7 @@ public class Scheduler implements Runnable { } for (ShardInfo completedShard : completedShards) { - final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifier()); + final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifierSerOpt()); if (createOrGetShardSyncTaskManager(streamIdentifier).syncShardAndLeaseInfo()) { log.info("Found completed shard, initiated new ShardSyncTak for " + completedShard.toString()); } @@ -635,7 +635,7 @@ public class Scheduler implements Runnable { checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. - final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifier()); + final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt()); // Irrespective of single stream app or multi stream app, streamConfig should always be available. final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); Validate.notNull(streamConfig, "StreamConfig should not be empty"); @@ -712,7 +712,7 @@ public class Scheduler implements Runnable { private StreamIdentifier getStreamIdentifier(Optional streamIdentifierString) { final StreamIdentifier streamIdentifier; if(streamIdentifierString.isPresent()) { - streamIdentifier = StreamIdentifier.fromString(streamIdentifierString.get()); + streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierString.get()); } else { Validate.isTrue(!isMultiStreamMode, "Should not be in MultiStream Mode"); streamIdentifier = this.currentStreamConfigMap.values().iterator().next().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 1102574f..f4143581 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 @@ -29,6 +29,7 @@ import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; +import com.google.common.annotations.VisibleForTesting; import lombok.Data; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; @@ -162,7 +163,7 @@ public class HierarchicalShardSyncer { throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamLeases = new ArrayList<>(); for (Lease lease : leaseRefresher.listLeases()) { - if (streamIdentifier.toString().equals(((MultiStreamLease)lease).streamIdentifier())) { + if (streamIdentifier.serialize().equals(((MultiStreamLease)lease).streamIdentifier())) { streamLeases.add(lease); } } @@ -761,7 +762,7 @@ public class HierarchicalShardSyncer { private static Lease newKCLMultiStreamLease(final Shard shard, final StreamIdentifier streamIdentifier) { MultiStreamLease newLease = new MultiStreamLease(); - newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.toString(), shard.shardId())); + newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), shard.shardId())); List parentShardIds = new ArrayList<>(2); if (shard.parentShardId() != null) { parentShardIds.add(shard.parentShardId()); @@ -771,7 +772,7 @@ public class HierarchicalShardSyncer { } newLease.parentShardIds(parentShardIds); newLease.ownerSwitchesSinceCheckpoint(0L); - newLease.streamIdentifier(streamIdentifier.toString()); + newLease.streamIdentifier(streamIdentifier.serialize()); newLease.shardId(shard.shardId()); return newLease; } @@ -857,7 +858,8 @@ public class HierarchicalShardSyncer { @Data @Accessors(fluent = true) - private static class MultiStreamArgs { + @VisibleForTesting + static class MultiStreamArgs { private final Boolean isMultiStreamMode; private final StreamIdentifier streamIdentifier; } 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 53751375..0c495558 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 @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; @@ -78,7 +77,7 @@ public class KinesisShardDetector implements ShardDetector { public KinesisShardDetector(KinesisAsyncClient kinesisClient, String streamName, long listShardsBackoffTimeInMillis, int maxListShardsRetryAttempts, long listShardsCacheAllowedAgeInSeconds, int maxCacheMissesBeforeReload, int cacheMissWarningModulus) { - this(kinesisClient, StreamIdentifier.fromStreamName(streamName), listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, + this(kinesisClient, StreamIdentifier.singleStreamInstance(streamName), listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); } 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 3361e3fb..2a5a0b1e 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 @@ -273,6 +273,12 @@ public class LeaseManagementConfig { return hierarchicalShardSyncer; } + /** + * Vends HierarchicalShardSyncer based on MultiStreamingMode. With MultiStreamMode shard syncer creates + * leases to accommodate more than one stream. + * @param isMultiStreamingMode + * @return HierarchicalShardSyncer + */ public HierarchicalShardSyncer hierarchicalShardSyncer(boolean isMultiStreamingMode) { if(hierarchicalShardSyncer == null) { hierarchicalShardSyncer = new HierarchicalShardSyncer(isMultiStreamingMode); @@ -313,6 +319,12 @@ public class LeaseManagementConfig { return leaseManagementFactory; } + /** + * Vends LeaseManagementFactory that performs serde based on leaseSerializer and shard sync based on isMultiStreamingMode + * @param leaseSerializer + * @param isMultiStreamingMode + * @return LeaseManagementFactory + */ public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer, boolean isMultiStreamingMode) { if(leaseManagementFactory == null) { leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), @@ -345,17 +357,14 @@ public class LeaseManagementConfig { return leaseManagementFactory; } + /** + * Set leaseManagementFactory and return the current LeaseManagementConfig instance. + * @param leaseManagementFactory + * @return LeaseManagementConfig + */ public LeaseManagementConfig leaseManagementFactory(final LeaseManagementFactory leaseManagementFactory) { this.leaseManagementFactory = leaseManagementFactory; return this; } -// private InitialPositionInStreamExtended getInitialPositionExtendedForStream(String streamName) { -// return multiStreamTracker() == null ? -// initialPositionInStream() : -// multiStreamTracker().initialPositionInStreamExtended(streamName) == null ? -// initialPositionInStream() : -// multiStreamTracker().initialPositionInStreamExtended(streamName); -// } - } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index 36bc5dd1..c4b2968c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -37,7 +37,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @ToString public class ShardInfo { - private final Optional streamIdentifier; + private final Optional streamIdentifierSerOpt; private final String shardId; private final String concurrencyToken; // Sorted list of parent shardIds. @@ -63,11 +63,20 @@ public class ShardInfo { this(shardId, concurrencyToken, parentShardIds, checkpoint, null); } + /** + * Creates a new ShardInfo object that has an option to pass a serialized streamIdentifier. + * The checkpoint is not part of the equality, but is used for debugging output. + * @param shardId + * @param concurrencyToken + * @param parentShardIds + * @param checkpoint + * @param streamIdentifierSer + */ public ShardInfo(@NonNull final String shardId, final String concurrencyToken, final Collection parentShardIds, final ExtendedSequenceNumber checkpoint, - final String streamIdentifier) { + final String streamIdentifierSer) { this.shardId = shardId; this.concurrencyToken = concurrencyToken; this.parentShardIds = new LinkedList<>(); @@ -78,7 +87,7 @@ public class ShardInfo { // This makes it easy to check for equality in ShardInfo.equals method. Collections.sort(this.parentShardIds); this.checkpoint = checkpoint; - this.streamIdentifier = Optional.ofNullable(streamIdentifier); + this.streamIdentifierSerOpt = Optional.ofNullable(streamIdentifierSer); } /** @@ -105,7 +114,7 @@ public class ShardInfo { @Override public int hashCode() { return new HashCodeBuilder() - .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamIdentifier.orElse("")).toHashCode(); + .append(concurrencyToken).append(parentShardIds).append(shardId).append(streamIdentifierSerOpt.orElse("")).toHashCode(); } /** @@ -130,18 +139,18 @@ public class ShardInfo { ShardInfo other = (ShardInfo) obj; return new EqualsBuilder().append(concurrencyToken, other.concurrencyToken) .append(parentShardIds, other.parentShardIds).append(shardId, other.shardId) - .append(streamIdentifier.orElse(""), other.streamIdentifier.orElse("")).isEquals(); + .append(streamIdentifierSerOpt.orElse(""), other.streamIdentifierSerOpt.orElse("")).isEquals(); } /** - * + * Utility method to derive lease key from ShardInfo * @param shardInfo - * @return + * @return lease key */ public static String getLeaseKey(ShardInfo shardInfo) { - return shardInfo.streamIdentifier().isPresent() ? - MultiStreamLease.getLeaseKey(shardInfo.streamIdentifier().get(), shardInfo.shardId()) : + return shardInfo.streamIdentifierSerOpt().isPresent() ? + MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardInfo.shardId()) : shardInfo.shardId(); } 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 f8a11c0a..3b7057de 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 @@ -378,7 +378,11 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { return leases.stream().map(DynamoDBLeaseCoordinator::convertLeaseToAssignment).collect(Collectors.toList()); } - // TODO : Halo : Check for better way + /** + * Utility method to convert the basic lease or multistream lease to ShardInfo + * @param lease + * @return ShardInfo + */ public static ShardInfo convertLeaseToAssignment(final Lease lease) { if (lease instanceof MultiStreamLease) { return new ShardInfo(((MultiStreamLease) lease).shardId(), lease.concurrencyToken().toString(), lease.parentShardIds(), 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 e383a41b..8c09af52 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,9 +16,6 @@ package software.amazon.kinesis.leases.dynamodb; import java.time.Duration; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import java.util.concurrent.ExecutorService; import lombok.Data; @@ -26,8 +23,6 @@ import lombok.NonNull; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.awssdk.utils.CollectionUtils; -import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; @@ -331,7 +326,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode) { - this(kinesisClient, new StreamConfig(StreamIdentifier.fromStreamName(streamName), initialPositionInStream), dynamoDBClient, tableName, + this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName, workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, @@ -391,6 +386,35 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.streamConfig = streamConfig; } + /** + * Constructor. + * @param kinesisClient + * @param dynamoDBClient + * @param tableName + * @param workerIdentifier + * @param executorService + * @param failoverTimeMillis + * @param epsilonMillis + * @param maxLeasesForWorker + * @param maxLeasesToStealAtOneTime + * @param maxLeaseRenewalThreads + * @param cleanupLeasesUponShardCompletion + * @param ignoreUnexpectedChildShards + * @param shardSyncIntervalMillis + * @param consistentReads + * @param listShardsBackoffTimeMillis + * @param maxListShardsRetryAttempts + * @param maxCacheMissesBeforeReload + * @param listShardsCacheAllowedAgeInSeconds + * @param cacheMissWarningModulus + * @param initialLeaseTableReadCapacity + * @param initialLeaseTableWriteCapacity + * @param hierarchicalShardSyncer + * @param tableCreatorCallback + * @param dynamoDbRequestTimeout + * @param billingMode + * @param leaseSerializer + */ public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, final ExecutorService executorService, final long failoverTimeMillis, final long epsilonMillis, @@ -457,6 +481,12 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { metricsFactory); } + /** + * Create ShardSyncTaskManager from the streamConfig passed + * @param metricsFactory + * @param streamConfig + * @return ShardSyncTaskManager + */ @Override public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { return new ShardSyncTaskManager(this.createShardDetector(streamConfig), @@ -476,7 +506,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { tableCreatorCallback, dynamoDbRequestTimeout, billingMode); } - @Override @Deprecated + @Override + @Deprecated public ShardDetector createShardDetector() { return new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis, maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, @@ -487,7 +518,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * KinesisShardDetector supports reading from service only using streamName. Support for accountId and * stream creation epoch is yet to be provided. * @param streamConfig - * @return + * @return ShardDetector */ @Override public ShardDetector createShardDetector(StreamConfig streamConfig) { 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 e41f7e08..171687bc 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 @@ -12,9 +12,9 @@ import java.util.Map; public interface MultiStreamTracker { /** - * Returns the map of streams and its associated stream specific config. + * Returns the list of stream config, to be processed by the current application. * - * @return List of stream names + * @return List of StreamConfig */ List streamConfigList(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java index e16695b2..72a6d66f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/ShardRecordProcessorFactory.java @@ -29,7 +29,7 @@ public interface ShardRecordProcessorFactory { ShardRecordProcessor shardRecordProcessor(); /** - * Returns a new instance of the ShardRecordProcessor for a stream + * Returns a new instance of the ShardRecordProcessor for a stream identifier * @param streamIdentifier * @return ShardRecordProcessor */ 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 f26c0e13..98046b6b 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 @@ -94,7 +94,7 @@ public class RetrievalConfig { @NonNull String applicationName) { this.kinesisClient = kinesisAsyncClient; this.appStreamTracker = Either - .right(new StreamConfig(StreamIdentifier.fromStreamName(streamName), initialPositionInStreamExtended)); + .right(new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStreamExtended)); this.applicationName = applicationName; } 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 45679089..cbbcb483 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 @@ -85,6 +85,7 @@ public class FanOutConfig implements RetrievalSpecificConfig { return new FanOutRetrievalFactory(kinesisClient, streamName, this::getOrCreateConsumerArn); } + // TODO : Halo. Need Stream Specific ConsumerArn to be passed from Customer private String getOrCreateConsumerArn(String streamName) { if (consumerArn != null) { return consumerArn; 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 f655d8fd..f609c1d9 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 @@ -49,10 +49,10 @@ public class FanOutRetrievalFactory implements RetrievalFactory { @Override public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, final MetricsFactory metricsFactory) { - final Optional streamIdentifierStr = shardInfo.streamIdentifier(); + final Optional streamIdentifierStr = shardInfo.streamIdentifierSerOpt(); final String streamName; if(streamIdentifierStr.isPresent()) { - streamName = StreamIdentifier.fromString(streamIdentifierStr.get()).streamName(); + streamName = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()).streamName(); } else { streamName = defaultStreamName; } 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 db6d2aae..1ea833a3 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 @@ -74,10 +74,18 @@ public class KinesisDataFetcher { @Deprecated public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory) { - this(kinesisClient, StreamIdentifier.fromStreamName(streamName), shardId, maxRecords, metricsFactory, PollingConfig.DEFAULT_REQUEST_TIMEOUT); + this(kinesisClient, StreamIdentifier.singleStreamInstance(streamName), shardId, maxRecords, metricsFactory, PollingConfig.DEFAULT_REQUEST_TIMEOUT); } - // Changing the constructor directly as this is an internal API + /** + * Constructs KinesisDataFetcher. + * @param kinesisClient + * @param streamIdentifier + * @param shardId + * @param maxRecords + * @param metricsFactory + * @param maxFutureWait + */ public KinesisDataFetcher(KinesisAsyncClient kinesisClient, StreamIdentifier streamIdentifier, String shardId, int maxRecords, MetricsFactory metricsFactory, Duration maxFutureWait) { this.kinesisClient = kinesisClient; this.streamIdentifier = streamIdentifier; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java index 93718f7c..73273c34 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java @@ -63,9 +63,9 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory { @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { - final StreamIdentifier streamIdentifier = shardInfo.streamIdentifier().isPresent() ? - StreamIdentifier.fromString(shardInfo.streamIdentifier().get()) : - StreamIdentifier.fromStreamName(streamName); + final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ? + StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) : + StreamIdentifier.singleStreamInstance(streamName); return new SynchronousGetRecordsRetrievalStrategy( new KinesisDataFetcher(kinesisClient, streamIdentifier, shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java index e73ad928..4a8c5250 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java @@ -68,9 +68,9 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, @NonNull final MetricsFactory metricsFactory) { - final StreamIdentifier streamIdentifier = shardInfo.streamIdentifier().isPresent() ? - StreamIdentifier.fromString(shardInfo.streamIdentifier().get()) : - StreamIdentifier.fromStreamName(streamName); + final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ? + StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) : + StreamIdentifier.singleStreamInstance(streamName); return new SynchronousGetRecordsRetrievalStrategy( new KinesisDataFetcher(kinesisClient, streamIdentifier, shardInfo.shardId(), maxRecords, metricsFactory, maxFutureWait)); 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 7cfc86a5..bd8c28e8 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -22,8 +22,10 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doCallRealMethod; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; @@ -32,14 +34,20 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.internal.verification.VerificationModeFactory.atMost; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.RejectedExecutionException; +import java.util.stream.Collectors; import io.reactivex.plugins.RxJavaPlugins; +import lombok.RequiredArgsConstructor; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,8 +60,11 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.KinesisClientLibException; import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -63,6 +74,8 @@ import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; import software.amazon.kinesis.lifecycle.events.InitializationInput; @@ -73,6 +86,7 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.Checkpointer; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShardRecordProcessor; @@ -124,6 +138,11 @@ public class SchedulerTest { private Checkpointer checkpoint; @Mock private WorkerStateChangeListener workerStateChangeListener; + @Mock + private MultiStreamTracker multiStreamTracker; + + private Map shardSyncTaskManagerMap = new HashMap<>(); + private Map shardDetectorMap = new HashMap<>(); @Before public void setup() { @@ -132,13 +151,25 @@ public class SchedulerTest { checkpointConfig = new CheckpointConfig().checkpointFactory(new TestKinesisCheckpointFactory()); coordinatorConfig = new CoordinatorConfig(applicationName).parentShardPollIntervalMillis(100L).workerStateChangeListener(workerStateChangeListener); leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, - workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory()); + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, false)); lifecycleConfig = new LifecycleConfig(); metricsConfig = new MetricsConfig(cloudWatchClient, namespace); processorConfig = new ProcessorConfig(shardRecordProcessorFactory); retrievalConfig = new RetrievalConfig(kinesisClient, streamName, applicationName) .retrievalFactory(retrievalFactory); + final List streamConfigList = new ArrayList() {{ + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc1:stream2:2"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream1:1"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + add(new StreamConfig(StreamIdentifier.multiStreamInstance("acc2:stream2:3"), InitialPositionInStreamExtended.newInitialPosition( + InitialPositionInStream.LATEST))); + }}; + + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(MetricsFactory.class))).thenReturn(recordsPublisher); @@ -245,7 +276,10 @@ public class SchedulerTest { public final void testInitializationFailureWithRetries() throws Exception { doNothing().when(leaseCoordinator).initialize(); when(shardDetector.listShards()).thenThrow(new RuntimeException()); - + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true)); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); scheduler.run(); verify(shardDetector, times(coordinatorConfig.maxInitializationAttempts())).listShards(); @@ -255,6 +289,8 @@ public class SchedulerTest { public final void testInitializationFailureWithRetriesWithConfiguredMaxInitializationAttempts() throws Exception { final int maxInitializationAttempts = 5; coordinatorConfig.maxInitializationAttempts(maxInitializationAttempts); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); @@ -267,6 +303,76 @@ public class SchedulerTest { verify(shardDetector, times(maxInitializationAttempts)).listShards(); } + @Test + public final void testMultiStreamInitialization() throws ProvisionedThroughputException, DependencyException { + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + scheduler.initialize(); + shardDetectorMap.values().stream() + .forEach(shardDetector -> verify(shardDetector, times(1)).listShards()); + } + + @Test + public final void testMultiStreamInitializationWithFailures() { + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, false)); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + scheduler.initialize(); + // Note : As of today we retry for all streams in the next attempt. Hence the retry for each stream will vary. + // At the least we expect 2 retries for each stream. Since there are 4 streams, we expect at most + // the number of calls to be 5. + shardDetectorMap.values().stream() + .forEach(shardDetector -> verify(shardDetector, atLeast(2)).listShards()); + shardDetectorMap.values().stream() + .forEach(shardDetector -> verify(shardDetector, atMost(5)).listShards()); + } + + + @Test + public final void testMultiStreamConsumersAreBuiltOncePerAccountStreamShard() throws KinesisClientLibException { + final String shardId = "shardId-000000000000"; + final String concurrencyToken = "concurrencyToken"; + final ExtendedSequenceNumber firstSequenceNumber = ExtendedSequenceNumber.TRIM_HORIZON; + final ExtendedSequenceNumber secondSequenceNumber = new ExtendedSequenceNumber("1000"); + final ExtendedSequenceNumber finalSequenceNumber = new ExtendedSequenceNumber("2000"); + + final List initialShardInfo = multiStreamTracker.streamConfigList().stream() + .map(sc -> new ShardInfo(shardId, concurrencyToken, null, firstSequenceNumber, + sc.streamIdentifier().serialize())).collect(Collectors.toList()); + final List firstShardInfo = multiStreamTracker.streamConfigList().stream() + .map(sc -> new ShardInfo(shardId, concurrencyToken, null, secondSequenceNumber, + sc.streamIdentifier().serialize())).collect(Collectors.toList()); + final List secondShardInfo = multiStreamTracker.streamConfigList().stream() + .map(sc -> new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber, + sc.streamIdentifier().serialize())).collect(Collectors.toList()); + + final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null); + + when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo); + when(checkpoint.getCheckpointObject(anyString())).thenReturn(firstCheckpoint); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig); + Scheduler schedulerSpy = spy(scheduler); + schedulerSpy.runProcessLoop(); + schedulerSpy.runProcessLoop(); + schedulerSpy.runProcessLoop(); + + initialShardInfo.stream().forEach( + shardInfo -> verify(schedulerSpy).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory))); + firstShardInfo.stream().forEach( + shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory))); + secondShardInfo.stream().forEach( + shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory))); + + } + @Test public final void testSchedulerShutdown() { scheduler.shutdown(); @@ -508,7 +614,12 @@ public class SchedulerTest { } + @RequiredArgsConstructor private class TestKinesisLeaseManagementFactory implements LeaseManagementFactory { + + private final boolean shardSyncFirstAttemptFailure; + private final boolean shouldReturnDefaultShardSyncTaskmanager; + @Override public LeaseCoordinator createLeaseCoordinator(MetricsFactory metricsFactory) { return leaseCoordinator; @@ -522,6 +633,19 @@ public class SchedulerTest { @Override public ShardSyncTaskManager createShardSyncTaskManager(MetricsFactory metricsFactory, StreamConfig streamConfig) { + if(shouldReturnDefaultShardSyncTaskmanager) { + return shardSyncTaskManager; + } + final ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); + final ShardDetector shardDetector = mock(ShardDetector.class); + shardSyncTaskManagerMap.put(streamConfig.streamIdentifier(), shardSyncTaskManager); + shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector); + when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + if(shardSyncFirstAttemptFailure) { + when(shardDetector.listShards()) + .thenThrow(new RuntimeException("Service Exception")) + .thenReturn(Collections.EMPTY_LIST); + } return shardSyncTaskManager; } @@ -537,7 +661,7 @@ public class SchedulerTest { @Override public ShardDetector createShardDetector(StreamConfig streamConfig) { - return shardDetector; + return shardDetectorMap.get(streamConfig.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 4dc3fdd3..0cc50c2a 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 @@ -55,6 +55,7 @@ import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.leases.exceptions.DependencyException; @@ -74,6 +75,10 @@ public class HierarchicalShardSyncerTest { private static final int EXPONENT = 128; private static final String LEASE_OWNER = "TestOwnere"; private static final MetricsScope SCOPE = new NullMetricsScope(); + private static final boolean MULTISTREAM_MODE_ON = true; + private static final String STREAM_IDENTIFIER = "acc:stream:1"; + private static final HierarchicalShardSyncer.MultiStreamArgs MULTI_STREAM_ARGS = new HierarchicalShardSyncer.MultiStreamArgs( + MULTISTREAM_MODE_ON, StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); private final boolean cleanupLeasesOfCompletedShards = true; private final boolean ignoreUnexpectedChildShards = false; @@ -95,6 +100,11 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer = new HierarchicalShardSyncer(); } + private void setupMultiStream() { + hierarchicalShardSyncer = new HierarchicalShardSyncer(true); + when(shardDetector.streamIdentifier()).thenReturn(StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); + } + /** * Test determineNewLeasesToCreate() where there are no shards */ @@ -107,6 +117,18 @@ public class HierarchicalShardSyncerTest { equalTo(true)); } + /** + * Test determineNewLeasesToCreate() where there are no shards for MultiStream + */ + @Test public void testDetermineNewLeasesToCreateNoShardsForMultiStream() { + final List shards = Collections.emptyList(); + final List leases = Collections.emptyList(); + + assertThat(HierarchicalShardSyncer + .determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS) + .isEmpty(), equalTo(true)); + } + /** * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed */ @@ -129,6 +151,29 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); } + /** + * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed + */ + @Test + public void testDetermineNewLeasesToCreate0Leases0ReshardsForMultiStream() { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); + final List currentLeases = Collections.emptyList(); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS); + final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set expectedLeaseIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); + + assertThat(newLeases.size(), equalTo(expectedLeaseIds.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseIds)); + } + /** * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but * one of the shards was marked as inconsistent. @@ -155,6 +200,33 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); } + /** + * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but + * one of the shards was marked as inconsistent. + */ + @Test + public void testDetermineNewLeasesToCreate0Leases0Reshards1InconsistentMultiStream() { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + final String shardId2 = "shardId-2"; + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); + final List currentLeases = Collections.emptyList(); + + final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); + final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set expectedLeaseShardIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); + assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + } + /** * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) */ @@ -208,6 +280,45 @@ public class HierarchicalShardSyncerTest { } + @Test + public void testCheckAndCreateLeasesForShardsIfMissingAtLatestMultiStream() throws Exception { + final List shards = constructShardListForGraphA(); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + + when(shardDetector.listShards()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + setupMultiStream(); + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + cleanupLeasesOfCompletedShards, false, SCOPE); + + 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)); + + extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); + + verify(shardDetector).listShards(); + verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + + } + + private List toMultiStreamLeaseList(List shardIdBasedLeases) { + return shardIdBasedLeases.stream().map(s -> STREAM_IDENTIFIER + ":" + s) + .collect(Collectors.toList()); + } + /** * Test checkAndCreateLeaseForNewShards with a pre-fetched list of shards. In this scenario, shardDetector.listShards() * should never be called. @@ -244,6 +355,42 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } + /** + * Test checkAndCreateLeaseForNewShards with a pre-fetched list of shards. In this scenario, shardDetector.listShards() + * should never be called. + */ + @Test + public void testCheckAndCreateLeasesForShardsWithShardListMultiStream() throws Exception { + final List latestShards = constructShardListForGraphA(); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + when(shardDetector.listShards()).thenReturn(latestShards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + setupMultiStream(); + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); + + 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)); + + extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); + + verify(shardDetector, never()).listShards(); + verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + } + /** * Test checkAndCreateLeaseForNewShards with an empty list of shards. In this scenario, shardDetector.listShards() * should never be called. @@ -306,6 +453,26 @@ public class HierarchicalShardSyncerTest { } } + @Test(expected = KinesisClientLibIOException.class) + public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenForMultiStream() throws Exception { + final List shards = new ArrayList<>(constructShardListForGraphA()); + final SequenceNumberRange range = shards.get(0).sequenceNumberRange().toBuilder().endingSequenceNumber(null) + .build(); + final Shard shard = shards.get(3).toBuilder().sequenceNumberRange(range).build(); + shards.remove(3); + shards.add(3, shard); + + when(shardDetector.listShards()).thenReturn(shards); + setupMultiStream(); + try { + hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, + INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, false, SCOPE); + } finally { + verify(shardDetector).listShards(); + verify(dynamoDBLeaseRefresher, never()).listLeases(); + } + } + /** * Test checkAndCreateLeasesForNewShards() when a parent is open and children of open parents are being ignored. */ @@ -354,6 +521,51 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } + @Test + public void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildrenMultiStream() throws Exception { + final List shards = new ArrayList<>(constructShardListForGraphA()); + final Shard shard = shards.get(5); + assertThat(shard.shardId(), equalTo("shardId-5")); + + shards.remove(5); + + // shardId-5 in graph A has two children (shardId-9 and shardId-10). if shardId-5 + // is not closed, those children should be ignored when syncing shards, no leases + // should be obtained for them, and we should obtain a lease on the still-open + // parent. + shards.add(5, + shard.toBuilder() + .sequenceNumberRange(shard.sequenceNumberRange().toBuilder().endingSequenceNumber(null).build()) + .build()); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + + when(shardDetector.listShards()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); + setupMultiStream(); + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + cleanupLeasesOfCompletedShards, true, SCOPE); + + final List leases = leaseCaptor.getAllValues(); + final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set leaseSequenceNumbers = leases.stream().map(Lease::checkpoint) + .collect(Collectors.toSet()); + + final Set expectedShardIds = new HashSet<>(toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-5", "shardId-8"))); + + assertThat(leaseKeys.size(), equalTo(expectedShardIds.size())); + assertThat(leaseKeys, equalTo(expectedShardIds)); + assertThat(leaseSequenceNumbers.size(), equalTo(1)); + + leaseSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); + + verify(shardDetector).listShards(); + verify(dynamoDBLeaseRefresher, times(expectedShardIds.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); + } + @Test public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShard() throws Exception { testCheckAndCreateLeasesForNewShardsAndClosedShard(ExtendedSequenceNumber.TRIM_HORIZON, @@ -711,6 +923,11 @@ public class HierarchicalShardSyncerTest { return createLeasesFromShards(Collections.singletonList(shard), checkpoint, leaseOwner).get(0); } + private MultiStreamLease createMultiStreamLeaseFromShard(final Shard shard, final ExtendedSequenceNumber checkpoint, + final String leaseOwner) { + return createMultiStreamLeasesFromShards(Collections.singletonList(shard), checkpoint, leaseOwner).get(0); + } + private List createLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, final String leaseOwner) { return shards.stream().map(shard -> { @@ -726,6 +943,29 @@ public class HierarchicalShardSyncerTest { }).collect(Collectors.toList()); } + private List createMultiStreamLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, + final String leaseOwner) { + return shards.stream().map(shard -> { + final Set parentShardIds = new HashSet<>(); + if (StringUtils.isNotEmpty(shard.parentShardId())) { + parentShardIds.add(shard.parentShardId()); + } + if (StringUtils.isNotEmpty(shard.adjacentParentShardId())) { + parentShardIds.add(shard.adjacentParentShardId()); + } + final MultiStreamLease msLease = new MultiStreamLease(); + msLease.shardId(shard.shardId()); + msLease.leaseOwner(leaseOwner); + msLease.leaseCounter(0L); + msLease.concurrencyToken(UUID.randomUUID()); + msLease.lastCounterIncrementNanos(0L); + msLease.checkpoint(checkpoint); + msLease.parentShardIds(parentShardIds); + msLease.streamIdentifier(STREAM_IDENTIFIER); + return msLease; + }).collect(Collectors.toList()); + } + @Test public void testCleanUpGarbageLeaseForNonExistentShard() throws Exception { final List shards = constructShardListForGraphA(); @@ -755,6 +995,35 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); } + @Test + public void testCleanUpGarbageLeaseForNonExistentShardForMultiStream() throws Exception { + final List shards = constructShardListForGraphA(); + final String garbageShardId = "shardId-garbage-001"; + final Shard garbageShard = ShardObjectHelper.newShard(garbageShardId, null, null, + ShardObjectHelper.newSequenceNumberRange("101", null)); + final Lease garbageLease = createMultiStreamLeaseFromShard(garbageShard, new ExtendedSequenceNumber("99"), LEASE_OWNER); + final List leases = new ArrayList<>( + createMultiStreamLeasesFromShards(shards, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER)); + leases.add(garbageLease); + + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); + + when(shardDetector.listShards()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(leases); + doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); + setupMultiStream(); + hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, + INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + + assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); + assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); + + verify(shardDetector, times(2)).listShards(); + verify(dynamoDBLeaseRefresher).listLeases(); + verify(dynamoDBLeaseRefresher).deleteLease(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); + } + private void testCheckAndCreateLeasesForShardsIfMissing(InitialPositionInStreamExtended initialPosition) throws Exception { final String shardId0 = "shardId-0"; 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 810f0159..23fb5dad 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 @@ -57,8 +57,6 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; -import javax.swing.*; - @RunWith(MockitoJUnitRunner.class) public class ConsumerStatesTest { private static final String STREAM_NAME = "TestStream"; @@ -115,7 +113,7 @@ public class ConsumerStatesTest { @Before public void setup() { - argument = new ShardConsumerArgument(shardInfo, StreamIdentifier.fromStreamName(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, 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 cd3e8af8..21228c75 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 @@ -31,7 +31,6 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.metrics.MetricsFactory; @@ -67,7 +66,8 @@ public class FanOutConfigTest { .streamName(TEST_STREAM_NAME); RetrievalFactory retrievalFactory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); +// doReturn(Optional.of(StreamIdentifier.singleStreamInstance(TEST_STREAM_NAME).serialize())).when(shardInfo).streamIdentifier(); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); retrievalFactory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(retrievalFactory, not(nullValue())); verify(consumerRegistration).getOrCreateStreamConsumerArn(); @@ -93,7 +93,7 @@ public class FanOutConfigTest { .streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); @@ -108,7 +108,7 @@ public class FanOutConfigTest { .streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; @@ -122,7 +122,7 @@ public class FanOutConfigTest { .consumerName(TEST_CONSUMER_NAME).streamName(TEST_STREAM_NAME); RetrievalFactory factory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); - doReturn(Optional.of(StreamIdentifier.fromStreamName(TEST_STREAM_NAME).toString())).when(shardInfo).streamIdentifier(); + doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); From 0285789a24cbe8f88ec1f3ce597cbb5e5a97d3c6 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 16 Mar 2020 11:21:30 -0700 Subject: [PATCH 017/159] Addressed review comments --- .../kinesis/common/StreamIdentifier.java | 21 ++++++++++-------- .../kinesis/leases/MultiStreamLease.java | 22 ++----------------- 2 files changed, 14 insertions(+), 29 deletions(-) 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 0bdce7fb..f4cbac29 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 @@ -3,6 +3,7 @@ package software.amazon.kinesis.common; import com.google.common.base.Joiner; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NonNull; import lombok.experimental.Accessors; import software.amazon.awssdk.utils.Validate; @@ -18,14 +19,16 @@ public class StreamIdentifier { private static final String DELIMITER = ":"; private static final Pattern PATTERN = Pattern.compile(".*" + ":" + ".*" + ":" + "[0-9]*"); - private StreamIdentifier(Optional accountIdOptional, String streamName, - Optional streamCreationEpochOptional) { - Validate.isTrue((accountIdOptional.isPresent() && streamCreationEpochOptional.isPresent()) || - (!accountIdOptional.isPresent() && !streamCreationEpochOptional.isPresent()), - "AccountId and StreamCreationEpoch must either be present together or not"); - this.accountIdOptional = accountIdOptional; + private StreamIdentifier(@NonNull String accountId, @NonNull String streamName, @NonNull Long streamCreationEpoch) { + this.accountIdOptional = Optional.of(accountId); this.streamName = streamName; - this.streamCreationEpochOptional = streamCreationEpochOptional; + this.streamCreationEpochOptional = Optional.of(streamCreationEpoch); + } + + private StreamIdentifier(@NonNull String streamName) { + this.accountIdOptional = Optional.empty(); + this.streamName = streamName; + this.streamCreationEpochOptional = Optional.empty(); } /** @@ -51,7 +54,7 @@ public class StreamIdentifier { public static StreamIdentifier multiStreamInstance(String streamIdentifierSer) { if (PATTERN.matcher(streamIdentifierSer).matches()) { final String[] split = streamIdentifierSer.split(DELIMITER); - return new StreamIdentifier(Optional.of(split[0]), split[1], Optional.of(Long.parseLong(split[2]))); + return new StreamIdentifier(split[0], split[1], Long.parseLong(split[2])); } else { throw new IllegalArgumentException("Unable to deserialize StreamIdentifier from " + streamIdentifierSer); } @@ -64,6 +67,6 @@ public class StreamIdentifier { */ public static StreamIdentifier singleStreamInstance(String streamName) { Validate.notEmpty(streamName, "StreamName should not be empty"); - return new StreamIdentifier(Optional.empty(), streamName, Optional.empty()); + return new StreamIdentifier(streamName); } } 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 862c203d..96818f74 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 @@ -1,5 +1,6 @@ package software.amazon.kinesis.leases; +import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.NonNull; @@ -15,6 +16,7 @@ import static com.google.common.base.Verify.verifyNotNull; @NoArgsConstructor @Getter @Accessors(fluent = true) +@EqualsAndHashCode(callSuper = true) public class MultiStreamLease extends Lease { @NonNull private String streamIdentifier; @@ -40,26 +42,6 @@ public class MultiStreamLease extends Lease { return streamIdentifier + ":" + shardId; } - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), streamIdentifier); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (!super.equals(obj)) { - return false; - } - if (!(obj instanceof MultiStreamLease)) { - return false; - } - MultiStreamLease other = (MultiStreamLease) obj; - return Objects.equals(streamIdentifier, other.streamIdentifier); - } - /** * Returns a deep copy of this object. Type-unsafe - there aren't good mechanisms for copy-constructing generics. * From f57a3326710d6f8fedf1a7c942f6505c5f51ce00 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 18 Mar 2020 11:56:20 -0700 Subject: [PATCH 018/159] Make periodic shard sync support multi streams --- .../coordinator/PeriodicShardSyncManager.java | 44 ++++++------ .../amazon/kinesis/coordinator/Scheduler.java | 56 ++++----------- .../kinesis/leases/ShardSyncTaskManager.java | 71 ++++++++++++++++--- .../kinesis/coordinator/SchedulerTest.java | 19 +++-- 4 files changed, 109 insertions(+), 81 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 7d166b98..5de09c37 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 @@ -18,16 +18,18 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; -import software.amazon.kinesis.leases.ShardSyncTask; +import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.lifecycle.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; -import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; -import software.amazon.kinesis.metrics.MetricsFactory; +import java.util.List; +import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; /** * The top level orchestrator for coordinating the periodic shard sync related @@ -42,21 +44,21 @@ class PeriodicShardSyncManager { private final String workerId; private final LeaderDecider leaderDecider; - private final ConsumerTask metricsEmittingShardSyncTask; + private final Map streamToShardSyncTaskManagerMap; private final ScheduledExecutorService shardSyncThreadPool; private boolean isRunning; - PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, MetricsFactory metricsFactory) { - this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory); + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, Map streamToShardSyncTaskManagerMap) { + this(workerId, leaderDecider, streamToShardSyncTaskManagerMap, Executors.newSingleThreadScheduledExecutor()); } - PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, ScheduledExecutorService shardSyncThreadPool, MetricsFactory metricsFactory) { + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, Map streamToShardSyncTaskManagerMap, + ScheduledExecutorService shardSyncThreadPool) { Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); - Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager."); this.workerId = workerId; this.leaderDecider = leaderDecider; - this.metricsEmittingShardSyncTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory); + this.streamToShardSyncTaskManagerMap = streamToShardSyncTaskManagerMap; this.shardSyncThreadPool = shardSyncThreadPool; } @@ -82,17 +84,14 @@ class PeriodicShardSyncManager { * Does not schedule periodic shardSync * @return the result of the task */ - public synchronized TaskResult syncShardsOnce() { - - Exception lastException = null; - try { - if (!isRunning) { - runShardSync(); + public synchronized void syncShardsOnce() throws Exception { + for (Map.Entry mapEntry : streamToShardSyncTaskManagerMap.entrySet()) { + final ShardSyncTaskManager shardSyncTaskManager = mapEntry.getValue(); + final TaskResult taskResult = shardSyncTaskManager.executeShardSyncTask(); + if (taskResult.getException() != null) { + throw taskResult.getException(); } - } catch (Exception e) { - lastException = e; } - return new TaskResult(lastException); } public void stop() { @@ -107,10 +106,11 @@ class PeriodicShardSyncManager { private void runShardSync() { if (leaderDecider.isLeader(workerId)) { - log.info(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); - final TaskResult taskResult = metricsEmittingShardSyncTask.call(); - if (taskResult != null && taskResult.getException() != null) { - throw new KinesisClientLibIOException("Failed to sync shards", taskResult.getException()); + for (Map.Entry mapEntry : streamToShardSyncTaskManagerMap.entrySet()) { + final ShardSyncTaskManager shardSyncTaskManager = mapEntry.getValue(); + if (!shardSyncTaskManager.syncShardAndLeaseInfo()) { + throw new KinesisClientLibIOException("Failed to submit shard sync task for stream " + shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + } } } else { log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); 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 5edf9472..dc730441 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 @@ -56,9 +56,7 @@ import software.amazon.kinesis.leases.LeaseSerializer; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardPrioritization; -import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.leases.ShardSyncTaskManager; -import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer; import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer; @@ -72,9 +70,7 @@ 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.MetricsCollectingTaskDecorator; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -85,19 +81,8 @@ import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; /** * @@ -136,7 +121,6 @@ public class Scheduler implements Runnable { private final LeaseCoordinator leaseCoordinator; private final Function shardSyncTaskManagerProvider; private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); - private final ShardSyncTaskManager shardSyncTaskManager; private final PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager; private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; @@ -278,7 +262,8 @@ public class Scheduler implements Runnable { // TODO : Halo : Check if this needs to be per stream. this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(isMultiStreamMode); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); - this.leaderElectedPeriodicShardSyncManager = buildPeriodicShardSyncManager(); + this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), + leaderDecider, streamToShardSyncTaskManagerMap); } /** @@ -320,25 +305,16 @@ public class Scheduler implements Runnable { log.info("Initializing LeaseCoordinator"); leaseCoordinator.initialize(); - TaskResult result; if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - waitUntilLeaseTableIsReady(); - for(Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { - final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); - log.info("Syncing Kinesis shard info for " + streamIdentifier); - final StreamConfig streamConfig = streamConfigEntry.getValue(); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamIdentifier), - leaseRefresher, streamConfig.initialPositionInStreamExtended(), - cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, - hierarchicalShardSyncer, metricsFactory); - result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); - // Throwing the exception, to prevent further syncs for other stream. - if (result.getException() != null) { - log.error("Caught exception when sync'ing info for " + streamIdentifier, result.getException()); - throw result.getException(); + if (!waitAndCheckIfLeaseTableIsReady()) { + for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); + createOrGetShardSyncTaskManager(streamIdentifier); + log.info("Creating shard sync task for " + streamIdentifier); } + leaderElectedPeriodicShardSyncManager.syncShardsOnce(); } } else { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); @@ -353,9 +329,10 @@ public class Scheduler implements Runnable { log.info("LeaseCoordinator is already running. No need to start it."); } log.info("Scheduling periodicShardSync)"); - // leaderElectedPeriodicShardSyncManager.start(); + // leaderElectedPeriodicShardSyncManager.start(shardSyncTasks); // TODO: enable periodicShardSync after https://github.com/jushkem/amazon-kinesis-client/pull/2 is merged - waitUntilHashRangeCovered(); + // TODO: Determine if waitUntilHashRangeCovered() is needed. + //waitUntilHashRangeCovered(); isDone = true; } catch (LeasingException e) { log.error("Caught exception when initializing LeaseCoordinator", e); @@ -382,7 +359,7 @@ public class Scheduler implements Runnable { } @VisibleForTesting - void waitUntilLeaseTableIsReady() throws InterruptedException, + boolean waitAndCheckIfLeaseTableIsReady() throws InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); long waitUntil = System.currentTimeMillis() + waitTime; @@ -393,6 +370,7 @@ public class Scheduler implements Runnable { log.info("Lease table is still empty. Checking again in {} ms", LEASE_TABLE_CHECK_FREQUENCY_MILLIS); Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS); } + return !leaseRefresher.isLeaseTableEmpty(); } private void waitUntilHashRangeCovered() throws InterruptedException { @@ -725,14 +703,6 @@ public class Scheduler implements Runnable { argument, lifecycleConfig.taskExecutionListener(), lifecycleConfig.readTimeoutsToIgnoreBeforeWarning()); } - private PeriodicShardSyncManager buildPeriodicShardSyncManager() { - final ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(getStreamIdentifier), leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, - metricsFactory); - return new PeriodicShardSyncManager(leaseManagementConfig.workerIdentifier(), - leaderDecider, shardSyncTask, metricsFactory); - } - /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. *

diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java index f6db72e3..8c8e0464 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java @@ -14,9 +14,12 @@ */ package software.amazon.kinesis.leases; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -53,6 +56,10 @@ public class ShardSyncTaskManager { private final HierarchicalShardSyncer hierarchicalShardSyncer; @NonNull private final MetricsFactory metricsFactory; + private ConsumerTask currentTask; + private CompletableFuture future; + private AtomicBoolean shardSyncRequestPending; + private final ReentrantLock lock; /** * Constructor. @@ -82,6 +89,8 @@ public class ShardSyncTaskManager { this.executorService = executorService; this.hierarchicalShardSyncer = new HierarchicalShardSyncer(); this.metricsFactory = metricsFactory; + this.shardSyncRequestPending = new AtomicBoolean(false); + this.lock = new ReentrantLock(); } /** @@ -110,16 +119,33 @@ public class ShardSyncTaskManager { this.executorService = executorService; this.hierarchicalShardSyncer = hierarchicalShardSyncer; this.metricsFactory = metricsFactory; + this.shardSyncRequestPending = new AtomicBoolean(false); + this.lock = new ReentrantLock(); } - private ConsumerTask currentTask; - private Future future; - - public synchronized boolean syncShardAndLeaseInfo() { - return checkAndSubmitNextTask(); + public TaskResult executeShardSyncTask() { + final ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, + leaseRefresher, + initialPositionInStream, + cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, + shardSyncIdleTimeMillis, + hierarchicalShardSyncer, + metricsFactory); + final ConsumerTask metricCollectingTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory); + return metricCollectingTask.call(); } - private synchronized boolean checkAndSubmitNextTask() { + public boolean syncShardAndLeaseInfo() { + try { + lock.lock(); + return checkAndSubmitNextTask(); + } finally { + lock.unlock(); + } + } + + private boolean checkAndSubmitNextTask() { boolean submittedNewTask = false; if ((future == null) || future.isCancelled() || future.isDone()) { if ((future != null) && future.isDone()) { @@ -145,18 +171,45 @@ public class ShardSyncTaskManager { hierarchicalShardSyncer, metricsFactory), metricsFactory); - future = executorService.submit(currentTask); + future = CompletableFuture.supplyAsync(() -> currentTask.call(), executorService) + .whenComplete((taskResult, exception) -> handlePendingShardSyncs(exception, taskResult)); submittedNewTask = true; if (log.isDebugEnabled()) { log.debug("Submitted new {} task.", currentTask.taskType()); } } else { if (log.isDebugEnabled()) { - log.debug("Previous {} task still pending. Not submitting new task.", currentTask.taskType()); + log.debug("Previous {} task still pending. Not submitting new task. " + + "Enqueued a request that will be executed when the current request completes.", currentTask.taskType()); } + shardSyncRequestPending.compareAndSet(false /*expected*/, true /*update*/); } - return submittedNewTask; } + private void handlePendingShardSyncs(Throwable exception, TaskResult taskResult) { + if (exception != null || taskResult.getException() != null) { + log.error("Caught exception running {} task: ", currentTask.taskType(), exception != null ? exception : taskResult.getException()); + } + // Acquire lock here. If shardSyncRequestPending is false in this completionStage and + // syncShardAndLeaseInfo is invoked, before completion stage exits (future completes) + // but right after the value of shardSyncRequestPending is checked, it will result in + // shardSyncRequestPending being set to true, but no pending futures to trigger the next + // ShardSyncTask. By executing this stage in a Reentrant lock, we ensure that if the + // previous task is in this completion stage, checkAndSubmitNextTask is not invoked + // until this completionStage exits. + try { + lock.lock(); + if (shardSyncRequestPending.get()) { + shardSyncRequestPending.set(false); + // reset future to null, so next call creates a new one + // without trying to get results from the old future. + future = null; + checkAndSubmitNextTask(); + } + } finally { + lock.unlock(); + } + } + } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index eabe25cd..b14f3303 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 @@ -78,6 +78,7 @@ import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; +import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.lifecycle.events.InitializationInput; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; @@ -175,6 +176,7 @@ public class SchedulerTest { when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + when(shardSyncTaskManager.executeShardSyncTask()).thenReturn(new TaskResult(null)); when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(MetricsFactory.class))).thenReturn(recordsPublisher); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, @@ -278,14 +280,14 @@ public class SchedulerTest { @Test public final void testInitializationFailureWithRetries() throws Exception { doNothing().when(leaseCoordinator).initialize(); - when(shardDetector.listShards()).thenThrow(new RuntimeException()); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenThrow(new RuntimeException()); leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, streamName, workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(false, true)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); scheduler.run(); - verify(shardDetector, times(coordinatorConfig.maxInitializationAttempts())).listShards(); + verify(dynamoDBLeaseRefresher, times(coordinatorConfig.maxInitializationAttempts())).isLeaseTableEmpty(); } @Test @@ -298,18 +300,20 @@ public class SchedulerTest { metricsConfig, processorConfig, retrievalConfig); doNothing().when(leaseCoordinator).initialize(); - when(shardDetector.listShards()).thenThrow(new RuntimeException()); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenThrow(new RuntimeException()); scheduler.run(); // verify initialization was retried for maxInitializationAttempts times - verify(shardDetector, times(maxInitializationAttempts)).listShards(); + verify(dynamoDBLeaseRefresher, times(maxInitializationAttempts)).isLeaseTableEmpty(); } @Test public final void testMultiStreamInitialization() throws ProvisionedThroughputException, DependencyException { retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) .retrievalFactory(retrievalFactory); + leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, true)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); scheduler.initialize(); @@ -322,7 +326,7 @@ public class SchedulerTest { retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) .retrievalFactory(retrievalFactory); leaseManagementConfig = new LeaseManagementConfig(tableName, dynamoDBClient, kinesisClient, - workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, false)); + workerIdentifier).leaseManagementFactory(new TestKinesisLeaseManagementFactory(true, true)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); scheduler.initialize(); @@ -388,7 +392,7 @@ public class SchedulerTest { when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); long startTime = System.currentTimeMillis(); - scheduler.waitUntilLeaseTableIsReady(); + scheduler.waitAndCheckIfLeaseTableIsReady(); long endTime = System.currentTimeMillis(); assertTrue(endTime - startTime > MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); @@ -407,7 +411,7 @@ public class SchedulerTest { when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); long startTime = System.currentTimeMillis(); - scheduler.waitUntilLeaseTableIsReady(); + scheduler.waitAndCheckIfLeaseTableIsReady(); long endTime = System.currentTimeMillis(); assertTrue(endTime - startTime < MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); @@ -681,6 +685,7 @@ public class SchedulerTest { shardSyncTaskManagerMap.put(streamConfig.streamIdentifier(), shardSyncTaskManager); shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + when(shardSyncTaskManager.executeShardSyncTask()).thenReturn(new TaskResult(null)); if(shardSyncFirstAttemptFailure) { when(shardDetector.listShards()) .thenThrow(new RuntimeException("Service Exception")) From 384fe5266c137144d4790e8507e6ba9002d74037 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 19 Mar 2020 15:47:27 -0700 Subject: [PATCH 019/159] Address comments --- .../amazon/kinesis/coordinator/Scheduler.java | 14 +++++++------- .../amazon/kinesis/coordinator/SchedulerTest.java | 4 ++-- 2 files changed, 9 insertions(+), 9 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 dc730441..85234f7f 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 @@ -94,7 +94,7 @@ 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 = 5 * 1000L; + private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; private static final long HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS = 5000L; private SchedulerLog slog = new SchedulerLog(); @@ -127,7 +127,6 @@ public class Scheduler implements Runnable { private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; private final GracefulShutdownCoordinator gracefulShutdownCoordinator; private final WorkerStateChangeListener workerStateChangeListener; - private final InitialPositionInStreamExtended initialPosition; private final MetricsFactory metricsFactory; private final long failoverTimeMillis; private final long taskBackoffTimeMillis; @@ -249,7 +248,6 @@ public class Scheduler implements Runnable { } this.leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); - this.initialPosition = retrievalConfig.initialPositionInStreamExtended(); this.failoverTimeMillis = this.leaseManagementConfig.failoverTimeMillis(); this.taskBackoffTimeMillis = this.lifecycleConfig.taskBackoffTimeMillis(); // this.retryGetRecordsInSeconds = this.retrievalConfig.retryGetRecordsInSeconds(); @@ -308,7 +306,8 @@ public class Scheduler implements Runnable { if (!skipShardSyncAtWorkerInitializationIfLeasesExist || leaseRefresher.isLeaseTableEmpty()) { // TODO: Resume the shard sync from failed stream in the next attempt, to avoid syncing // TODO: for already synced streams - if (!waitAndCheckIfLeaseTableIsReady()) { + if (shouldInitiateLeaseSync()) { + log.info("Worker {} is initiating the lease sync.", leaseManagementConfig.workerIdentifier()); for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); createOrGetShardSyncTaskManager(streamIdentifier); @@ -359,18 +358,19 @@ public class Scheduler implements Runnable { } @VisibleForTesting - boolean waitAndCheckIfLeaseTableIsReady() throws InterruptedException, + boolean shouldInitiateLeaseSync() throws InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); long waitUntil = System.currentTimeMillis() + waitTime; - while (System.currentTimeMillis() < waitUntil && leaseRefresher.isLeaseTableEmpty()) { + boolean isLeaseTableEmpty = true; + while (System.currentTimeMillis() < waitUntil && (isLeaseTableEmpty = leaseRefresher.isLeaseTableEmpty())) { // check every 3 seconds if lease table is still empty, // to minimize contention between all workers bootstrapping at the same time log.info("Lease table is still empty. Checking again in {} ms", LEASE_TABLE_CHECK_FREQUENCY_MILLIS); Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS); } - return !leaseRefresher.isLeaseTableEmpty(); + return isLeaseTableEmpty; } private void waitUntilHashRangeCovered() throws InterruptedException { 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 b14f3303..812b05df 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 @@ -392,7 +392,7 @@ public class SchedulerTest { when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); long startTime = System.currentTimeMillis(); - scheduler.waitAndCheckIfLeaseTableIsReady(); + scheduler.shouldInitiateLeaseSync(); long endTime = System.currentTimeMillis(); assertTrue(endTime - startTime > MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); @@ -411,7 +411,7 @@ public class SchedulerTest { when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); long startTime = System.currentTimeMillis(); - scheduler.waitAndCheckIfLeaseTableIsReady(); + scheduler.shouldInitiateLeaseSync(); long endTime = System.currentTimeMillis(); assertTrue(endTime - startTime < MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); From bb495e4f60f564ffc71c33ab319a52fb582fb4ba Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Fri, 20 Mar 2020 11:54:40 -0700 Subject: [PATCH 020/159] Adding metric for ShardSyncTaskManager --- .../amazon/kinesis/coordinator/PeriodicShardSyncManager.java | 3 ++- .../java/software/amazon/kinesis/leases/ShardSyncTask.java | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) 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 5de09c37..1d02f47c 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 @@ -109,7 +109,8 @@ class PeriodicShardSyncManager { for (Map.Entry mapEntry : streamToShardSyncTaskManagerMap.entrySet()) { final ShardSyncTaskManager shardSyncTaskManager = mapEntry.getValue(); if (!shardSyncTaskManager.syncShardAndLeaseInfo()) { - throw new KinesisClientLibIOException("Failed to submit shard sync task for stream " + shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + log.warn("Failed to submit shard sync task for stream {}. This could be due to the previous shard sync task not finished.", + shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); } } } else { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java index c59608b2..b8f581c6 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java @@ -23,6 +23,7 @@ import software.amazon.kinesis.lifecycle.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.lifecycle.TaskType; import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.MetricsLevel; import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; @@ -62,6 +63,7 @@ public class ShardSyncTask implements ConsumerTask { public TaskResult call() { Exception exception = null; final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHARD_SYNC_TASK_OPERATION); + boolean shardSyncSuccess = true; try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, @@ -72,7 +74,9 @@ public class ShardSyncTask implements ConsumerTask { } catch (Exception e) { log.error("Caught exception while sync'ing Kinesis shards and leases", e); exception = e; + shardSyncSuccess = false; } finally { + MetricsUtil.addSuccess(scope, "SyncShards", shardSyncSuccess, MetricsLevel.DETAILED); MetricsUtil.endScope(scope); } From c00c943a79de68abcf630bc83afa07dfa0d44879 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 20 Mar 2020 17:19:04 -0700 Subject: [PATCH 021/159] MultiStream Sync and logging changes --- .../kinesis/common/DiagnosticUtils.java | 8 +- .../amazon/kinesis/common/StreamConfig.java | 1 + .../amazon/kinesis/coordinator/Scheduler.java | 133 +++++++++++++++--- .../leases/HierarchicalShardSyncer.java | 65 +++++---- .../lifecycle/BlockOnParentShardTask.java | 13 +- .../amazon/kinesis/lifecycle/ProcessTask.java | 27 ++-- .../kinesis/lifecycle/ShardConsumer.java | 22 +-- .../lifecycle/ShardConsumerSubscriber.java | 16 ++- .../kinesis/lifecycle/ShutdownTask.java | 14 +- .../kinesis/retrieval/RetrievalConfig.java | 3 +- .../fanout/FanOutRecordsPublisher.java | 126 +++++++++-------- .../retrieval/polling/KinesisDataFetcher.java | 14 +- .../polling/PrefetchRecordsPublisher.java | 41 +++--- .../leases/HierarchicalShardSyncerTest.java | 39 +++++ 14 files changed, 352 insertions(+), 170 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java index db0393e1..37eea1a3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/DiagnosticUtils.java @@ -28,11 +28,11 @@ public class DiagnosticUtils { /** * Util for RecordPublisher to measure the event delivery latency of the executor service and take appropriate action. - * @param shardId of the shard that is having delayed delivery + * @param resourceIdentifier of the shard that is having delayed delivery * @param enqueueTimestamp of the event submitted to the executor service * @param log Slf4j Logger from RecordPublisher to log the events */ - public static void takeDelayedDeliveryActionIfRequired(String shardId, Instant enqueueTimestamp, Logger log) { + public static void takeDelayedDeliveryActionIfRequired(String resourceIdentifier, Instant enqueueTimestamp, Logger log) { final long durationBetweenEnqueueAndAckInMillis = Duration .between(enqueueTimestamp, Instant.now()).toMillis(); if (durationBetweenEnqueueAndAckInMillis > MAX_TIME_BETWEEN_REQUEST_RESPONSE / 3) { @@ -41,9 +41,9 @@ public class DiagnosticUtils { "{}: Record delivery time to shard consumer is high at {} millis. Check the ExecutorStateEvent logs" + " to see the state of the executor service. Also check if the RecordProcessor's processing " + "time is high. ", - shardId, durationBetweenEnqueueAndAckInMillis); + resourceIdentifier, durationBetweenEnqueueAndAckInMillis); } else if (log.isDebugEnabled()) { - log.debug("{}: Record delivery time to shard consumer is {} millis", shardId, + log.debug("{}: Record delivery time to shard consumer is {} millis", resourceIdentifier, durationBetweenEnqueueAndAckInMillis); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java index 999182b6..3cf0eeb2 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 @@ -6,6 +6,7 @@ import lombok.experimental.Accessors; @Value @Accessors(fluent = true) public class StreamConfig { + // TODO: Consider having streamIdentifier as the unique identifier of this class. StreamIdentifier streamIdentifier; InitialPositionInStreamExtended initialPositionInStreamExtended; } 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 48467dfb..c8354eff 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 @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -35,6 +36,7 @@ import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; import io.reactivex.plugins.RxJavaPlugins; import lombok.AccessLevel; import lombok.Getter; @@ -45,6 +47,8 @@ import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; @@ -52,6 +56,7 @@ import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseSerializer; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardPrioritization; @@ -61,7 +66,10 @@ import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer; import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.LeasingException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; import software.amazon.kinesis.lifecycle.ShardConsumerArgument; @@ -74,6 +82,7 @@ import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.processor.Checkpointer; +import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; import software.amazon.kinesis.processor.ShutdownNotificationAware; @@ -89,6 +98,8 @@ import software.amazon.kinesis.retrieval.RetrievalConfig; @Slf4j public class Scheduler implements Runnable { + private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 10000L; + private SchedulerLog slog = new SchedulerLog(); private final CheckpointConfig checkpointConfig; @@ -111,8 +122,8 @@ public class Scheduler implements Runnable { private final DiagnosticEventHandler diagnosticEventHandler; // private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private final LeaseCoordinator leaseCoordinator; - private final Function shardSyncTaskManagerProvider; - private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); + private final Function shardSyncTaskManagerProvider; + private final Map streamToShardSyncTaskManagerMap = new HashMap<>(); private final ShardPrioritization shardPrioritization; private final boolean cleanupLeasesUponShardCompletion; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; @@ -122,11 +133,13 @@ public class Scheduler implements Runnable { private final long failoverTimeMillis; private final long taskBackoffTimeMillis; private final boolean isMultiStreamMode; + // TODO : halo : make sure we generate streamConfig if entry not present. private final Map currentStreamConfigMap; + private final MultiStreamTracker multiStreamTracker; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; - private final Function shardDetectorProvider; + private final Function shardDetectorProvider; private final boolean ignoreUnexpetedChildShards; private final AggregatorUtil aggregatorUtil; private final HierarchicalShardSyncer hierarchicalShardSyncer; @@ -142,6 +155,9 @@ public class Scheduler implements Runnable { private final Object lock = new Object(); + private Stopwatch streamSyncWatch = Stopwatch.createUnstarted(); + private boolean leasesSyncedOnAppInit = false; + /** * Used to ensure that only one requestedShutdown is in progress at a time. */ @@ -190,6 +206,9 @@ public class Scheduler implements Runnable { .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)), streamConfig -> Collections.singletonMap(streamConfig.streamIdentifier(), streamConfig)); + this.multiStreamTracker = this.retrievalConfig.appStreamTracker().map( + multiStreamTracker -> multiStreamTracker, + streamConfig -> null); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); // Determine leaseSerializer based on availability of MultiStreamTracker. @@ -217,9 +236,9 @@ public class Scheduler implements Runnable { this.diagnosticEventHandler = new DiagnosticEventLogger(); // TODO : Halo : Handle case of no StreamConfig present in streamConfigList() for the supplied streamName. // TODO : Pass the immutable map here instead of using mst.streamConfigList() - this.shardSyncTaskManagerProvider = streamIdentifier -> this.leaseManagementConfig + this.shardSyncTaskManagerProvider = streamConfig -> this.leaseManagementConfig .leaseManagementFactory(leaseSerializer, isMultiStreamMode) - .createShardSyncTaskManager(this.metricsFactory, this.currentStreamConfigMap.get(streamIdentifier)); + .createShardSyncTaskManager(this.metricsFactory, streamConfig); this.shardPrioritization = this.coordinatorConfig.shardPrioritization(); this.cleanupLeasesUponShardCompletion = this.leaseManagementConfig.cleanupLeasesUponShardCompletion(); this.skipShardSyncAtWorkerInitializationIfLeasesExist = @@ -242,7 +261,7 @@ public class Scheduler implements Runnable { // this.maxGetRecordsThreadPool = this.retrievalConfig.maxGetRecordsThreadPool(); this.listShardsBackoffTimeMillis = this.retrievalConfig.listShardsBackoffTimeInMillis(); this.maxListShardsRetryAttempts = this.retrievalConfig.maxListShardsRetryAttempts(); - this.shardDetectorProvider = streamIdentifier -> createOrGetShardSyncTaskManager(streamIdentifier).shardDetector(); + this.shardDetectorProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); // TODO : Halo : Check if this needs to be per stream. @@ -298,7 +317,7 @@ public class Scheduler implements Runnable { final StreamIdentifier streamIdentifier = streamConfigEntry.getKey(); log.info("Syncing Kinesis shard info for " + streamIdentifier); final StreamConfig streamConfig = streamConfigEntry.getValue(); - ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamIdentifier), + ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetectorProvider.apply(streamConfig), leaseRefresher, streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, 0L, hierarchicalShardSyncer, metricsFactory); @@ -321,6 +340,7 @@ public class Scheduler implements Runnable { } else { log.info("LeaseCoordinator is already running. No need to start it."); } + streamSyncWatch.start(); isDone = true; } catch (LeasingException e) { log.error("Caught exception when initializing LeaseCoordinator", e); @@ -364,14 +384,20 @@ public class Scheduler implements Runnable { for (ShardInfo completedShard : completedShards) { final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifierSerOpt()); - if (createOrGetShardSyncTaskManager(streamIdentifier).syncShardAndLeaseInfo()) { - log.info("Found completed shard, initiated new ShardSyncTak for " + completedShard.toString()); + final StreamConfig streamConfig = currentStreamConfigMap + .getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + if (createOrGetShardSyncTaskManager(streamConfig).syncShardAndLeaseInfo()) { + log.info("{} : Found completed shard, initiated new ShardSyncTak for {} ", + streamIdentifier.serialize(), completedShard.toString()); } } // clean up shard consumers for unassigned shards cleanupShardConsumers(assignedShards); + // check for new streams and sync with the scheduler state + checkAndSyncStreamShardsAndLeases(); + logExecutorState(); slog.info("Sleeping ..."); Thread.sleep(shardConsumerDispatchPollIntervalMillis); @@ -387,6 +413,77 @@ public class Scheduler implements Runnable { slog.resetInfoLogging(); } + + /** + * Note: This method has package level access solely for testing purposes. + * Sync all streams method. + * @return streams that are being synced by this worker + */ + private Set checkAndSyncStreamShardsAndLeases() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final Set streamsSynced = new HashSet<>(); + + if (isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS)) { + final Map newStreamConfigMap = new HashMap<>(); + // Making an immutable copy + newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() + .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); + + // This is done to ensure that we clean up the stale streams lingering in the lease table. + syncStreamsFromLeaseTableOnAppInit(); + + for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.syncShardAndLeaseInfo(); + currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); + streamsSynced.add(streamIdentifier); + } else { + if (log.isDebugEnabled()) { + log.debug(streamIdentifier + " is already being processed - skipping shard sync."); + } + } + } + + // TODO: Remove assumption that each Worker gets the full list of streams + Iterator currentStreamConfigIter = currentStreamConfigMap.keySet().iterator(); + while (currentStreamConfigIter.hasNext()) { + StreamIdentifier streamIdentifier = currentStreamConfigIter.next(); + if (!newStreamConfigMap.containsKey(streamIdentifier)) { + log.info("Found old/deleted stream: " + streamIdentifier + ". Syncing shards of that stream."); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(currentStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.syncShardAndLeaseInfo(); + currentStreamConfigIter.remove(); + streamsSynced.add(streamIdentifier); + } + } + streamSyncWatch.reset().start(); + } + return streamsSynced; + } + + private Set syncStreamsFromLeaseTableOnAppInit() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + if (!leasesSyncedOnAppInit && isMultiStreamMode) { + final Set streamIdentifiers = leaseCoordinator.leaseRefresher().listLeases().stream() + .map(lease -> StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())) + .collect(Collectors.toSet()); + for (StreamIdentifier streamIdentifier : streamIdentifiers) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + currentStreamConfigMap.put(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + } + } + leasesSyncedOnAppInit = true; + } + return Collections.emptySet(); + } + + // When a stream is no longer needed to be tracked, return a default StreamConfig with LATEST for faster shard end. + private StreamConfig getDefaultStreamConfig(StreamIdentifier streamIdentifier) { + return new StreamConfig(streamIdentifier, InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)); + } + /** * Returns whether worker can shutdown immediately. Note that this method is called from Worker's {{@link #run()} * method before every loop run, so method must do minimum amount of work to not impact shard processing timings. @@ -588,7 +685,8 @@ public class Scheduler implements Runnable { if (!firstItem) { builder.append(", "); } - builder.append(shardInfo.shardId()); + builder.append(shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) + .orElse(shardInfo.shardId())); firstItem = false; } slog.info("Current stream shard assignments: " + builder.toString()); @@ -624,8 +722,8 @@ public class Scheduler implements Runnable { return consumer; } - private ShardSyncTaskManager createOrGetShardSyncTaskManager(StreamIdentifier streamIdentifier) { - return streamToShardSyncTaskManagerMap.computeIfAbsent(streamIdentifier, s -> shardSyncTaskManagerProvider.apply(s)); + private ShardSyncTaskManager createOrGetShardSyncTaskManager(StreamConfig streamConfig) { + return streamToShardSyncTaskManagerMap.computeIfAbsent(streamConfig, s -> shardSyncTaskManagerProvider.apply(s)); } protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, @@ -637,8 +735,10 @@ public class Scheduler implements Runnable { // get the default stream name for the single stream application. final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt()); // Irrespective of single stream app or multi stream app, streamConfig should always be available. - final StreamConfig streamConfig = currentStreamConfigMap.get(streamIdentifier); - Validate.notNull(streamConfig, "StreamConfig should not be empty"); + // If we have a shardInfo, that is not present in currentStreamConfigMap for whatever reason, then return default stream config + // to gracefully complete the reading. + final StreamConfig streamConfig = currentStreamConfigMap.getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + Validate.notNull(streamConfig, "StreamConfig should not be null"); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, streamConfig.streamIdentifier(), leaseCoordinator, @@ -657,7 +757,7 @@ public class Scheduler implements Runnable { streamConfig.initialPositionInStreamExtended(), cleanupLeasesUponShardCompletion, ignoreUnexpetedChildShards, - shardDetectorProvider.apply(streamConfig.streamIdentifier()), + shardDetectorProvider.apply(streamConfig), aggregatorUtil, hierarchicalShardSyncer, metricsFactory); @@ -682,7 +782,8 @@ public class Scheduler implements Runnable { ShardConsumer consumer = shardInfoShardConsumerMap.get(shard); if (consumer.leaseLost()) { shardInfoShardConsumerMap.remove(shard); - log.debug("Removed consumer for {} as lease has been lost", shard.shardId()); + log.debug("Removed consumer for {} as lease has been lost", + shard.streamIdentifierSerOpt().map(s -> s + ":" + shard.shardId()).orElse(shard.shardId())); } else { consumer.executeLifecycle(); } 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 f4143581..6aaa81c4 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 @@ -24,6 +24,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; @@ -64,6 +65,8 @@ public class HierarchicalShardSyncer { private final boolean isMultiStreamMode; + private String streamIdentifier = ""; + public HierarchicalShardSyncer() { isMultiStreamMode = false; } @@ -99,6 +102,7 @@ public class HierarchicalShardSyncer { final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + this.streamIdentifier = shardDetector.streamIdentifier().serialize(); final List latestShards = getShardList(shardDetector); checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); @@ -110,8 +114,9 @@ public class HierarchicalShardSyncer { final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + this.streamIdentifier = shardDetector.streamIdentifier().serialize(); if (!CollectionUtils.isNullOrEmpty(latestShards)) { - log.debug("Num shards: {}", latestShards.size()); + log.debug("{} - Num shards: {}", streamIdentifier, latestShards.size()); } final Map shardIdToShardMap = constructShardIdToShardMap(latestShards); @@ -127,7 +132,7 @@ public class HierarchicalShardSyncer { final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamIdentifier()); final List newLeasesToCreate = determineNewLeasesToCreate(latestShards, currentLeases, initialPosition, inconsistentShardIds, multiStreamArgs); - log.debug("Num new leases to create: {}", newLeasesToCreate.size()); + log.debug("{} - Num new leases to create: {}", streamIdentifier, newLeasesToCreate.size()); for (Lease lease : newLeasesToCreate) { long startTime = System.currentTimeMillis(); boolean success = false; @@ -218,7 +223,7 @@ public class HierarchicalShardSyncer { for (String shardId : shardIdsOfClosedShards) { final Shard shard = shardIdToShardMap.get(shardId); if (shard == null) { - log.info("Shard {} is not present in Kinesis anymore.", shardId); + log.info("{} : Shard {} is not present in Kinesis anymore.", streamIdentifier, shardId); continue; } @@ -360,25 +365,26 @@ public class HierarchicalShardSyncer { final MultiStreamArgs multiStreamArgs) { final Map shardIdToNewLeaseMap = new HashMap<>(); final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); - + final String streamIdentifier = Optional.ofNullable(multiStreamArgs.streamIdentifier()) + .map(streamId -> streamId.serialize()).orElse(""); final Set shardIdsOfCurrentLeases = currentLeases.stream() - .peek(lease -> log.debug("Existing lease: {}", lease)) + .peek(lease -> log.debug("{} : Existing lease: {}", streamIdentifier, lease)) .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) .collect(Collectors.toSet()); - final List openShards = getOpenShards(shards); + final List openShards = getOpenShards(shards, streamIdentifier); final Map memoizationContext = new HashMap<>(); // Iterate over the open shards and find those that don't have any lease entries. for (Shard shard : openShards) { final String shardId = shard.shardId(); - log.debug("Evaluating leases for open shard {} and its ancestors.", shardId); + log.debug("{} : Evaluating leases for open shard {} and its ancestors.", streamIdentifier, shardId); if (shardIdsOfCurrentLeases.contains(shardId)) { - log.debug("Lease for shardId {} already exists. Not creating a lease", shardId); + log.debug("{} : Lease for shardId {} already exists. Not creating a lease", streamIdentifier, shardId); } else if (inconsistentShardIds.contains(shardId)) { - log.info("shardId {} is an inconsistent child. Not creating a lease", shardId); + log.info("{} : shardId {} is an inconsistent child. Not creating a lease", streamIdentifier, shardId); } else { - log.debug("Need to create a lease for shardId {}", shardId); + log.debug("{} : Need to create a lease for shardId {}", streamIdentifier, shardId); final Lease newLease = multiStreamArgs.isMultiStreamMode() ? newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : newKCLLease(shard); @@ -415,7 +421,7 @@ public class HierarchicalShardSyncer { } else { newLease.checkpoint(convertToCheckpoint(initialPosition)); } - log.debug("Set checkpoint of {} to {}", newLease.leaseKey(), newLease.checkpoint()); + log.debug("{} : Set checkpoint of {} to {}", streamIdentifier, newLease.leaseKey(), newLease.checkpoint()); shardIdToNewLeaseMap.put(shardId, newLease); } } @@ -464,7 +470,7 @@ public class HierarchicalShardSyncer { final Map shardIdToShardMapOfAllKinesisShards, final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext, final MultiStreamArgs multiStreamArgs) { - + final String streamIdentifier = getStreamIdentifier(multiStreamArgs); final Boolean previousValue = memoizationContext.get(shardId); if (previousValue != null) { return previousValue; @@ -489,9 +495,9 @@ public class HierarchicalShardSyncer { memoizationContext, multiStreamArgs)) { isDescendant = true; descendantParentShardIds.add(parentShardId); - log.debug("Parent shard {} is a descendant.", parentShardId); + log.debug("{} : Parent shard {} is a descendant.", streamIdentifier, parentShardId); } else { - log.debug("Parent shard {} is NOT a descendant.", parentShardId); + log.debug("{} : Parent shard {} is NOT a descendant.", streamIdentifier, parentShardId); } } @@ -499,7 +505,7 @@ public class HierarchicalShardSyncer { if (isDescendant) { for (String parentShardId : parentShardIds) { if (!shardIdsOfCurrentLeases.contains(parentShardId)) { - log.debug("Need to create a lease for shardId {}", parentShardId); + log.debug("{} : Need to create a lease for shardId {}", streamIdentifier, parentShardId); Lease lease = shardIdToLeaseMapOfNewShards.get(parentShardId); if (lease == null) { lease = multiStreamArgs.isMultiStreamMode() ? @@ -593,6 +599,7 @@ public class HierarchicalShardSyncer { final List trackedLeases, final LeaseRefresher leaseRefresher, final MultiStreamArgs multiStreamArgs) throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException { + final String streamIdentifier = getStreamIdentifier(multiStreamArgs); final Set kinesisShards = shards.stream().map(Shard::shardId).collect(Collectors.toSet()); // Check if there are leases for non-existent shards @@ -600,14 +607,15 @@ public class HierarchicalShardSyncer { .filter(lease -> isCandidateForCleanup(lease, kinesisShards, multiStreamArgs)).collect(Collectors.toList()); if (!CollectionUtils.isNullOrEmpty(garbageLeases)) { - log.info("Found {} candidate leases for cleanup. Refreshing list of" - + " Kinesis shards to pick up recent/latest shards", garbageLeases.size()); + log.info("{} : Found {} candidate leases for cleanup. Refreshing list of" + + " Kinesis shards to pick up recent/latest shards", streamIdentifier, garbageLeases.size()); final Set currentKinesisShardIds = getShardList(shardDetector).stream().map(Shard::shardId) .collect(Collectors.toSet()); for (Lease lease : garbageLeases) { if (isCandidateForCleanup(lease, currentKinesisShardIds, multiStreamArgs)) { - log.info("Deleting lease for shard {} as it is not present in Kinesis stream.", lease.leaseKey()); + log.info("{} : Deleting lease for shard {} as it is not present in Kinesis stream.", + streamIdentifier, lease.leaseKey()); leaseRefresher.deleteLease(lease); } } @@ -627,14 +635,16 @@ public class HierarchicalShardSyncer { static boolean isCandidateForCleanup(final Lease lease, final Set currentKinesisShardIds, final MultiStreamArgs multiStreamArgs) throws KinesisClientLibIOException { - boolean isCandidateForCleanup = true; + final String streamIdentifier = getStreamIdentifier(multiStreamArgs); + + boolean isCandidateForCleanup = true; final String shardId = shardIdFromLeaseDeducer.apply(lease, multiStreamArgs); if (currentKinesisShardIds.contains(shardId)) { isCandidateForCleanup = false; } else { - log.info("Found lease for non-existent shard: {}. Checking its parent shards", shardId); + log.info("{} : Found lease for non-existent shard: {}. Checking its parent shards", streamIdentifier, shardId); final Set parentShardIds = lease.parentShardIds(); for (String parentShardId : parentShardIds) { @@ -643,7 +653,7 @@ public class HierarchicalShardSyncer { if (currentKinesisShardIds.contains(parentShardId)) { final String message = String.format("Parent shard %s exists but not the child shard %s", parentShardId, shardId); - log.info(message); + log.info("{} : {}", streamIdentifier, message); throw new KinesisClientLibIOException(message); } } @@ -730,8 +740,8 @@ public class HierarchicalShardSyncer { } if (okayToDelete) { - log.info("Deleting lease for shard {} as it has been completely processed and processing of child " - + "shards has begun.", shardIdFromLeaseDeducer.apply(leaseForClosedShard, multiStreamArgs)); + log.info("{} : Deleting lease for shard {} as it has been completely processed and processing of child " + + "shards has begun.", streamIdentifier, shardIdFromLeaseDeducer.apply(leaseForClosedShard, multiStreamArgs)); leaseRefresher.deleteLease(leaseForClosedShard); } } @@ -794,9 +804,9 @@ public class HierarchicalShardSyncer { * @param allShards All shards returved via DescribeStream. We assume this to represent a consistent shard list. * @return List of open shards (shards at the tip of the stream) - may include shards that are not yet active. */ - static List getOpenShards(final List allShards) { + static List getOpenShards(final List allShards, final String streamIdentifier) { return allShards.stream().filter(shard -> shard.sequenceNumberRange().endingSequenceNumber() == null) - .peek(shard -> log.debug("Found open shard: {}", shard.shardId())).collect(Collectors.toList()); + .peek(shard -> log.debug("{} : Found open shard: {}", streamIdentifier, shard.shardId())).collect(Collectors.toList()); } private static ExtendedSequenceNumber convertToCheckpoint(final InitialPositionInStreamExtended position) { @@ -812,6 +822,11 @@ public class HierarchicalShardSyncer { return checkpoint; } + + private static String getStreamIdentifier(MultiStreamArgs multiStreamArgs) { + return Optional.ofNullable(multiStreamArgs.streamIdentifier()) + .map(streamId -> streamId.serialize()).orElse("single_stream_mode"); + } /** Helper class to compare leases based on starting sequence number of the corresponding shards. * diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java index 4ea4212e..37a092e8 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java @@ -54,7 +54,8 @@ public class BlockOnParentShardTask implements ConsumerTask { @Override public TaskResult call() { Exception exception = null; - + final String shardInfoId = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) + .orElse(shardInfo.shardId()); try { boolean blockedOnParentShard = false; for (String shardId : shardInfo.parentShardIds()) { @@ -62,20 +63,20 @@ public class BlockOnParentShardTask implements ConsumerTask { if (lease != null) { ExtendedSequenceNumber checkpoint = lease.checkpoint(); if ((checkpoint == null) || (!checkpoint.equals(ExtendedSequenceNumber.SHARD_END))) { - log.debug("Shard {} is not yet done. Its current checkpoint is {}", shardId, checkpoint); + log.debug("Shard {} is not yet done. Its current checkpoint is {}", shardInfoId, checkpoint); blockedOnParentShard = true; exception = new BlockedOnParentShardException("Parent shard not yet done"); break; } else { - log.debug("Shard {} has been completely processed.", shardId); + log.debug("Shard {} has been completely processed.", shardInfoId); } } else { - log.info("No lease found for shard {}. Not blocking on completion of this shard.", shardId); + log.info("No lease found for shard {}. Not blocking on completion of this shard.", shardInfoId); } } if (!blockedOnParentShard) { - log.info("No need to block on parents {} of shard {}", shardInfo.parentShardIds(), shardInfo.shardId()); + log.info("No need to block on parents {} of shard {}", shardInfo.parentShardIds(), shardInfoId); return new TaskResult(null); } } catch (Exception e) { @@ -85,7 +86,7 @@ public class BlockOnParentShardTask implements ConsumerTask { try { Thread.sleep(parentShardPollIntervalMillis); } catch (InterruptedException e) { - log.error("Sleep interrupted when waiting on parent shard(s) of {}", shardInfo.shardId(), e); + log.error("Sleep interrupted when waiting on parent shard(s) of {}", shardInfoId, e); } return new TaskResult(exception); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java index 6c223650..fd036c9f 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 @@ -60,6 +60,7 @@ public class ProcessTask implements ConsumerTask { private final ProcessRecordsInput processRecordsInput; private final MetricsFactory metricsFactory; private final AggregatorUtil aggregatorUtil; + private final String shardInfoId; public ProcessTask(@NonNull ShardInfo shardInfo, @NonNull ShardRecordProcessor shardRecordProcessor, @@ -74,6 +75,8 @@ public class ProcessTask implements ConsumerTask { @NonNull AggregatorUtil aggregatorUtil, @NonNull MetricsFactory metricsFactory) { this.shardInfo = shardInfo; + this.shardInfoId = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) + .orElse(shardInfo.shardId()); this.shardRecordProcessor = shardRecordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; this.backoffTimeMillis = backoffTimeMillis; @@ -121,7 +124,7 @@ public class ProcessTask implements ConsumerTask { } if (processRecordsInput.isAtShardEnd() && processRecordsInput.records().isEmpty()) { - log.info("Reached end of shard {} and have no records to process", shardInfo.shardId()); + log.info("Reached end of shard {} and have no records to process", shardInfoId); return new TaskResult(null, true); } @@ -142,13 +145,13 @@ public class ProcessTask implements ConsumerTask { } success = true; } catch (RuntimeException e) { - log.error("ShardId {}: Caught exception: ", shardInfo.shardId(), e); + log.error("ShardId {}: Caught exception: ", shardInfoId, e); exception = e; backoff(); } if (processRecordsInput.isAtShardEnd()) { - log.info("Reached end of shard {}, and processed {} records", shardInfo.shardId(), processRecordsInput.records().size()); + log.info("Reached end of shard {}, and processed {} records", shardInfoId, processRecordsInput.records().size()); return new TaskResult(null, true); } return new TaskResult(exception); @@ -174,7 +177,7 @@ public class ProcessTask implements ConsumerTask { try { Thread.sleep(this.backoffTimeMillis); } catch (InterruptedException ie) { - log.debug("{}: Sleep was interrupted", shardInfo.shardId(), ie); + log.debug("{}: Sleep was interrupted", shardInfoId, ie); } } @@ -188,7 +191,7 @@ public class ProcessTask implements ConsumerTask { */ private void callProcessRecords(ProcessRecordsInput input, List records) { log.debug("Calling application processRecords() with {} records from {}", records.size(), - shardInfo.shardId()); + shardInfoId); final ProcessRecordsInput processRecordsInput = ProcessRecordsInput.builder().records(records).cacheExitTime(input.cacheExitTime()).cacheEntryTime(input.cacheEntryTime()) .checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build(); @@ -200,8 +203,8 @@ public class ProcessTask implements ConsumerTask { shardRecordProcessor.processRecords(processRecordsInput); } catch (Exception e) { log.error("ShardId {}: Application processRecords() threw an exception when processing shard ", - shardInfo.shardId(), e); - log.error("ShardId {}: Skipping over the following data records: {}", shardInfo.shardId(), records); + shardInfoId, e); + log.error("ShardId {}: Skipping over the following data records: {}", shardInfoId, records); } finally { MetricsUtil.addLatency(scope, RECORD_PROCESSOR_PROCESS_RECORDS_METRIC, startTime, MetricsLevel.SUMMARY); MetricsUtil.endScope(scope); @@ -226,17 +229,17 @@ public class ProcessTask implements ConsumerTask { * the time when the task started */ private void handleNoRecords(long startTimeMillis) { - log.debug("Kinesis didn't return any records for shard {}", shardInfo.shardId()); + log.debug("Kinesis didn't return any records for shard {}", shardInfoId); long sleepTimeMillis = idleTimeInMilliseconds - (System.currentTimeMillis() - startTimeMillis); if (sleepTimeMillis > 0) { sleepTimeMillis = Math.max(sleepTimeMillis, idleTimeInMilliseconds); try { log.debug("Sleeping for {} ms since there were no new records in shard {}", sleepTimeMillis, - shardInfo.shardId()); + shardInfoId); Thread.sleep(sleepTimeMillis); } catch (InterruptedException e) { - log.debug("ShardId {}: Sleep was interrupted", shardInfo.shardId()); + log.debug("ShardId {}: Sleep was interrupted", shardInfoId); } } } @@ -273,8 +276,8 @@ public class ProcessTask implements ConsumerTask { if (extendedSequenceNumber.compareTo(lastCheckpointValue) <= 0) { recordIterator.remove(); - log.debug("removing record with ESN {} because the ESN is <= checkpoint ({})", extendedSequenceNumber, - lastCheckpointValue); + log.debug("{} : removing record with ESN {} because the ESN is <= checkpoint ({})", shardInfoId, + extendedSequenceNumber, lastCheckpointValue); continue; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java index 99a680bf..e34f2ea4 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 @@ -62,6 +62,7 @@ public class ShardConsumer { private final Function taskMetricsDecorator; private final int bufferSize; private final TaskExecutionListener taskExecutionListener; + private final String streamIdentifier; private ConsumerTask currentTask; private TaskOutcome taskOutcome; @@ -124,6 +125,7 @@ public class ShardConsumer { this.recordsPublisher = recordsPublisher; this.executorService = executorService; this.shardInfo = shardInfo; + this.streamIdentifier = shardInfo.streamIdentifierSerOpt().orElse("single_stream_mode"); this.shardConsumerArgument = shardConsumerArgument; this.logWarningForTaskAfterMillis = logWarningForTaskAfterMillis; this.taskExecutionListener = taskExecutionListener; @@ -208,8 +210,8 @@ public class ShardConsumer { } Throwable dispatchFailure = subscriber.getAndResetDispatchFailure(); if (dispatchFailure != null) { - log.warn("Exception occurred while dispatching incoming data. The incoming data has been skipped", - dispatchFailure); + log.warn("{} : Exception occurred while dispatching incoming data. The incoming data has been skipped", + streamIdentifier, dispatchFailure); return dispatchFailure; } @@ -238,7 +240,7 @@ public class ShardConsumer { Instant now = Instant.now(); Duration timeSince = Duration.between(subscriber.lastDataArrival(), now); if (timeSince.toMillis() > value) { - log.warn("Last time data arrived: {} ({})", lastDataArrival, timeSince); + log.warn("{} : Last time data arrived: {} ({})", streamIdentifier, lastDataArrival, timeSince); } } }); @@ -250,11 +252,11 @@ public class ShardConsumer { if (taken != null) { String message = longRunningTaskMessage(taken); if (log.isDebugEnabled()) { - log.debug("{} Not submitting new task.", message); + log.debug("{} : {} Not submitting new task.", streamIdentifier, message); } logWarningForTaskAfterMillis.ifPresent(value -> { if (taken.toMillis() > value) { - log.warn(message); + log.warn("{} : {}", streamIdentifier, message); } }); } @@ -358,7 +360,7 @@ public class ShardConsumer { nextState = currentState.failureTransition(); break; default: - log.error("No handler for outcome of {}", outcome.name()); + log.error("{} : No handler for outcome of {}", streamIdentifier, outcome.name()); nextState = currentState.failureTransition(); break; } @@ -382,9 +384,9 @@ public class ShardConsumer { Exception taskException = taskResult.getException(); if (taskException instanceof BlockedOnParentShardException) { // No need to log the stack trace for this exception (it is very specific). - log.debug("Shard {} is blocked on completion of parent shard.", shardInfo.shardId()); + log.debug("{} : Shard {} is blocked on completion of parent shard.", streamIdentifier, shardInfo.shardId()); } else { - log.debug("Caught exception running {} task: ", currentTask.taskType(), taskResult.getException()); + log.debug("{} : Caught exception running {} task: ", streamIdentifier, currentTask.taskType(), taskResult.getException()); } } } @@ -411,10 +413,10 @@ public class ShardConsumer { * @return true if shutdown is complete (false if shutdown is still in progress) */ public boolean leaseLost() { - log.debug("Shutdown({}): Lease lost triggered.", shardInfo.shardId()); + log.debug("{} : Shutdown({}): Lease lost triggered.", streamIdentifier, shardInfo.shardId()); if (subscriber != null) { subscriber.cancel(); - log.debug("Shutdown({}): Subscriber cancelled.", shardInfo.shardId()); + log.debug("{} : Shutdown({}): Subscriber cancelled.", streamIdentifier, shardInfo.shardId()); } markForShutdown(ShutdownReason.LEASE_LOST); return isShutdown(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java index 4c05ac94..177c0f43 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 @@ -40,8 +40,8 @@ class ShardConsumerSubscriber implements Subscriber { private final int bufferSize; private final ShardConsumer shardConsumer; private final int readTimeoutsToIgnoreBeforeWarning; + private final String shardInfoId; private volatile int readTimeoutSinceLastRead = 0; - @VisibleForTesting final Object lockObject = new Object(); // This holds the last time an attempt of request to upstream service was made including the first try to @@ -70,6 +70,8 @@ class ShardConsumerSubscriber implements Subscriber { this.bufferSize = bufferSize; this.shardConsumer = shardConsumer; this.readTimeoutsToIgnoreBeforeWarning = readTimeoutsToIgnoreBeforeWarning; + this.shardInfoId = shardConsumer.shardInfo().streamIdentifierSerOpt() + .map(s -> s + ":" + shardConsumer.shardInfo().shardId()).orElse(shardConsumer.shardInfo().shardId()); } @@ -107,7 +109,7 @@ class ShardConsumerSubscriber implements Subscriber { if (retrievalFailure != null) { synchronized (lockObject) { String logMessage = String.format("%s: Failure occurred in retrieval. Restarting data requests", - shardConsumer.shardInfo().shardId()); + shardInfoId); if (retrievalFailure instanceof RetryableRetrievalException) { log.debug(logMessage, retrievalFailure.getCause()); } else { @@ -130,7 +132,7 @@ class ShardConsumerSubscriber implements Subscriber { if (timeSinceLastResponse.toMillis() > maxTimeBetweenRequests) { log.error( "{}: Last request was dispatched at {}, but no response as of {} ({}). Cancelling subscription, and restarting. Last successful request details -- {}", - shardConsumer.shardInfo().shardId(), lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails()); + shardInfoId, lastRequestTime, now, timeSinceLastResponse, recordsPublisher.getLastSuccessfulRequestDetails()); cancel(); // Start the subscription again which will update the lastRequestTime as well. @@ -157,7 +159,7 @@ class ShardConsumerSubscriber implements Subscriber { subscription); } catch (Throwable t) { - log.warn("{}: Caught exception from handleInput", shardConsumer.shardInfo().shardId(), t); + log.warn("{}: Caught exception from handleInput", shardInfoId, t); synchronized (lockObject) { dispatchFailure = t; } @@ -193,7 +195,7 @@ class ShardConsumerSubscriber implements Subscriber { log.warn( "{}: onError(). Cancelling subscription, and marking self as failed. KCL will " + "recreate the subscription as neccessary to continue processing.", - shardConsumer.shardInfo().shardId(), t); + shardInfoId, t); } protected void logOnErrorReadTimeoutWarning(Throwable t) { @@ -202,13 +204,13 @@ class ShardConsumerSubscriber implements Subscriber { + "are seeing this warning frequently consider increasing the SDK timeouts " + "by providing an OverrideConfiguration to the kinesis client. Alternatively you" + "can configure LifecycleConfig.readTimeoutsToIgnoreBeforeWarning to suppress" - + "intermittant ReadTimeout warnings.", shardConsumer.shardInfo().shardId(), t); + + "intermittant ReadTimeout warnings.", shardInfoId, t); } @Override public void onComplete() { log.debug("{}: onComplete(): Received onComplete. Activity should be triggered externally", - shardConsumer.shardInfo().shardId()); + shardInfoId); } public void cancel() { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 2bfcd358..81b954a6 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 @@ -82,6 +82,8 @@ public class ShutdownTask implements ConsumerTask { private final TaskType taskType = TaskType.SHUTDOWN; + private String shardInfoId = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) + .orElse(shardInfo.shardId()); /* * Invokes ShardRecordProcessor shutdown() API. * (non-Javadoc) @@ -112,7 +114,7 @@ public class ShutdownTask implements ConsumerTask { if (CollectionUtils.isNullOrEmpty(latestShards) || !isShardInContextParentOfAny(latestShards)) { localReason = ShutdownReason.LEASE_LOST; dropLease(); - log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfo.shardId()); + log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfoId); } } @@ -124,7 +126,7 @@ public class ShutdownTask implements ConsumerTask { } log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", - shardInfo.shardId(), shardInfo.concurrencyToken(), localReason); + shardInfoId, shardInfo.concurrencyToken(), localReason); final ShutdownInput shutdownInput = ShutdownInput.builder().shutdownReason(localReason) .checkpointer(recordProcessorCheckpointer).build(); final long startTime = System.currentTimeMillis(); @@ -135,7 +137,7 @@ public class ShutdownTask implements ConsumerTask { if (lastCheckpointValue == null || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfo.shardId() + ". Application must checkpoint upon shard end. " + + + shardInfoId + ". Application must checkpoint upon shard end. " + "See ShardRecordProcessor.shardEnded javadocs for more information."); } } else { @@ -143,7 +145,7 @@ public class ShutdownTask implements ConsumerTask { } log.debug("Shutting down retrieval strategy."); recordsPublisher.shutdown(); - log.debug("Record processor completed shutdown() for shard {}", shardInfo.shardId()); + log.debug("Record processor completed shutdown() for shard {}", shardInfoId); } catch (Exception e) { applicationException = true; throw e; @@ -152,11 +154,11 @@ public class ShutdownTask implements ConsumerTask { } if (localReason == ShutdownReason.SHARD_END) { - log.debug("Looking for child shards of shard {}", shardInfo.shardId()); + log.debug("Looking for child shards of shard {}", shardInfoId); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); - log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); + log.debug("Finished checking for child shards of shard {}", shardInfoId); } return new TaskResult(null); 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 98046b6b..746fdc19 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 @@ -105,7 +105,7 @@ public class RetrievalConfig { this.applicationName = applicationName; } - public void initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) { + public RetrievalConfig initialPositionInStreamExtended(InitialPositionInStreamExtended initialPositionInStreamExtended) { final StreamConfig[] streamConfig = new StreamConfig[1]; this.appStreamTracker.apply(multiStreamTracker -> { throw new IllegalArgumentException( @@ -113,6 +113,7 @@ public class RetrievalConfig { }, sc -> streamConfig[0] = sc); this.appStreamTracker = Either .right(new StreamConfig(streamConfig[0].streamIdentifier(), initialPositionInStreamExtended)); + return this; } public RetrievalFactory retrievalFactory() { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index 27cad136..c24a3803 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 @@ -51,7 +51,6 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.time.Instant; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; @@ -60,7 +59,6 @@ import java.util.stream.Collectors; import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired; -@RequiredArgsConstructor @Slf4j @KinesisClientInternalApi public class FanOutRecordsPublisher implements RecordsPublisher { @@ -73,7 +71,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private final KinesisAsyncClient kinesis; private final String shardId; private final String consumerArn; - + private final String streamAndShardId; private final Object lockObject = new Object(); private final AtomicInteger subscribeToShardId = new AtomicInteger(0); @@ -91,11 +89,25 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private RequestDetails lastSuccessfulRequestDetails = new RequestDetails(); + public FanOutRecordsPublisher(KinesisAsyncClient kinesis, String shardId, String consumerArn) { + this.kinesis = kinesis; + this.shardId = shardId; + this.consumerArn = consumerArn; + this.streamAndShardId = shardId; + } + + public FanOutRecordsPublisher(KinesisAsyncClient kinesis, String shardId, String consumerArn, String streamIdentifierSer) { + this.kinesis = kinesis; + this.shardId = shardId; + this.consumerArn = consumerArn; + this.streamAndShardId = streamIdentifierSer + ":" + shardId; + } + @Override public void start(ExtendedSequenceNumber extendedSequenceNumber, InitialPositionInStreamExtended initialPositionInStreamExtended) { synchronized (lockObject) { - log.debug("[{}] Initializing Publisher @ Sequence: {} -- Initial Position: {}", shardId, + log.debug("[{}] Initializing Publisher @ Sequence: {} -- Initial Position: {}", streamAndShardId, extendedSequenceNumber, initialPositionInStreamExtended); this.initialPositionInStreamExtended = initialPositionInStreamExtended; this.currentSequenceNumber = extendedSequenceNumber.sequenceNumber(); @@ -174,7 +186,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // It is now safe to remove the element recordsDeliveryQueue.poll(); // Take action based on the time spent by the event in queue. - takeDelayedDeliveryActionIfRequired(shardId, recordsRetrievedContext.getEnqueueTimestamp(), log); + takeDelayedDeliveryActionIfRequired(streamAndShardId, recordsRetrievedContext.getEnqueueTimestamp(), log); // Update current sequence number for the successfully delivered event. currentSequenceNumber = ((FanoutRecordsRetrieved)recordsRetrieved).continuationSequenceNumber(); // Update the triggering flow for post scheduling upstream request. @@ -190,13 +202,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow != null && recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier() .equals(flow.getSubscribeToShardId())) { log.error( - "{}: Received unexpected ack for the active subscription {}. Throwing. ", - shardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); + "{}: Received unexpected ack for the active subscription {}. Throwing. ", streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); throw new IllegalStateException("Unexpected ack for the active subscription"); } // Otherwise publisher received a stale ack. else { - log.info("{}: Publisher received an ack for stale subscription {}. Ignoring.", shardId, + log.info("{}: Publisher received an ack for stale subscription {}. Ignoring.", streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier().getFlowIdentifier()); } } @@ -219,10 +230,10 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (IllegalStateException e) { log.warn("{}: Unable to enqueue the payload due to capacity restrictions in delivery queue with remaining capacity {}. Last successful request details -- {}", - shardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails); + streamAndShardId, recordsDeliveryQueue.remainingCapacity(), lastSuccessfulRequestDetails); throw e; } catch (Throwable t) { - log.error("{}: Unable to deliver event to the shard consumer.", shardId, t); + log.error("{}: Unable to deliver event to the shard consumer.", streamAndShardId, t); throw t; } } @@ -290,7 +301,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { String instanceId = shardId + "-" + subscribeInvocationId; log.debug( "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#subscribeToShard) @ {} id: {} -- Starting subscribe to shard", - shardId, connectionStart, instanceId); + streamAndShardId, connectionStart, instanceId); flow = new RecordFlow(this, connectionStart, instanceId); kinesis.subscribeToShard(request, flow); } @@ -303,12 +314,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if(hasValidFlow()) { log.warn( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- Subscriber is null." + - " Last successful request details -- {}", shardId, flow.connectionStartedAt, + " Last successful request details -- {}", streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId, lastSuccessfulRequestDetails); } else { log.warn( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) -- Subscriber and flow are null." + - " Last successful request details -- {}", shardId, lastSuccessfulRequestDetails); + " Last successful request details -- {}", streamAndShardId, lastSuccessfulRequestDetails); } return; } @@ -320,8 +331,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow != null) { String logMessage = String.format( "%s: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ %s id: %s -- %s." + - " Last successful request details -- %s", - shardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails); + " Last successful request details -- %s", streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId, category.throwableTypeString, lastSuccessfulRequestDetails); switch (category.throwableType) { case READ_TIMEOUT: log.debug(logMessage, propagationThrowable); @@ -339,13 +349,14 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } flow.cancel(); } - log.debug("{}: availableQueueSpace zeroing from {}", shardId, availableQueueSpace); + log.debug("{}: availableQueueSpace zeroing from {}", streamAndShardId, availableQueueSpace); availableQueueSpace = 0; try { handleFlowError(propagationThrowable, triggeringFlow); } catch (Throwable innerThrowable) { - log.warn("{}: Exception while calling subscriber.onError. Last successful request details -- {}", shardId, lastSuccessfulRequestDetails, innerThrowable); + log.warn("{}: Exception while calling subscriber.onError. Last successful request details -- {}", + streamAndShardId, lastSuccessfulRequestDetails, innerThrowable); } subscriber = null; flow = null; @@ -353,7 +364,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (triggeringFlow != null) { log.debug( "{}: [SubscriptionLifetime] - (FanOutRecordsPublisher#errorOccurred) @ {} id: {} -- {} -> triggeringFlow wasn't the active flow. Didn't dispatch error", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId, + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId, category.throwableTypeString); triggeringFlow.cancel(); } @@ -367,7 +378,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // Clear any lingering records in the queue. if (!recordsDeliveryQueue.isEmpty()) { log.warn("{}: Found non-empty queue while starting subscription. This indicates unsuccessful clean up of" - + "previous subscription - {}. Last successful request details -- {}", shardId, subscribeToShardId, lastSuccessfulRequestDetails); + + "previous subscription - {}. Last successful request details -- {}", streamAndShardId, subscribeToShardId, lastSuccessfulRequestDetails); recordsDeliveryQueue.clear(); } } @@ -383,7 +394,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (t.getCause() instanceof ResourceNotFoundException) { log.debug( "{}: Could not call SubscribeToShard successfully because shard no longer exists. Marking shard for completion.", - shardId); + streamAndShardId); // The ack received for this onNext event will be ignored by the publisher as the global flow object should // be either null or renewed when the ack's flow identifier is evaluated. FanoutRecordsRetrieved response = new FanoutRecordsRetrieved( @@ -452,7 +463,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (!hasValidSubscriber()) { log.debug( "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Subscriber is null.", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); triggeringFlow.cancel(); if (flow != null) { flow.cancel(); @@ -462,7 +473,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (!isActiveFlow(triggeringFlow)) { log.debug( "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Received records for an inactive flow.", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); return; } @@ -478,7 +489,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { bufferCurrentEventAndScheduleIfRequired(recordsRetrieved, triggeringFlow); } catch (Throwable t) { log.warn("{}: Unable to buffer or schedule onNext for subscriber. Failing publisher." + - " Last successful request details -- {}", shardId, lastSuccessfulRequestDetails); + " Last successful request details -- {}", streamAndShardId, lastSuccessfulRequestDetails); errorOccurred(triggeringFlow, t); } } @@ -488,7 +499,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (availableQueueSpace <= 0) { log.debug( "{}: [SubscriptionLifetime] (FanOutRecordsPublisher#recordsReceived) @ {} id: {} -- Attempted to decrement availableQueueSpace to below 0", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); } else { availableQueueSpace--; if (availableQueueSpace > 0) { @@ -503,12 +514,13 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private void onComplete(RecordFlow triggeringFlow) { synchronized (lockObject) { - log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", shardId, + log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); triggeringFlow.cancel(); if (!hasValidSubscriber()) { - log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", shardId, + log.debug("{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {}", + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); return; } @@ -516,15 +528,15 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (!isActiveFlow(triggeringFlow)) { log.debug( "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher#onComplete) @ {} id: {} -- Received spurious onComplete from unexpected flow. Ignoring.", - shardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); + streamAndShardId, triggeringFlow.connectionStartedAt, triggeringFlow.subscribeToShardId); return; } if (currentSequenceNumber != null) { - log.debug("{}: Shard hasn't ended. Resubscribing.", shardId); + log.debug("{}: Shard hasn't ended. Resubscribing.", streamAndShardId); subscribeToShard(currentSequenceNumber); } else { - log.debug("{}: Shard has ended completing subscriber.", shardId); + log.debug("{}: Shard has ended completing subscriber.", streamAndShardId); subscriber.onComplete(); } } @@ -536,7 +548,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (subscriber != null) { log.error( "{}: A subscribe occurred while there was an active subscriber. Sending error to current subscriber", - shardId); + streamAndShardId); MultipleSubscriberException multipleSubscriberException = new MultipleSubscriberException(); // @@ -575,7 +587,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (subscriber != s) { log.warn( "{}: (FanOutRecordsPublisher/Subscription#request) - Rejected an attempt to request({}), because subscribers don't match. Last successful request details -- {}", - shardId, n, lastSuccessfulRequestDetails); + streamAndShardId, n, lastSuccessfulRequestDetails); return; } if (flow == null) { @@ -584,7 +596,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // log.debug( "{}: (FanOutRecordsPublisher/Subscription#request) - Request called for a null flow.", - shardId); + streamAndShardId); errorOccurred(flow, new IllegalStateException("Attempted to request on a null flow.")); return; } @@ -602,19 +614,19 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (subscriber != s) { log.warn( "{}: (FanOutRecordsPublisher/Subscription#cancel) - Rejected attempt to cancel subscription, because subscribers don't match. Last successful request details -- {}", - shardId, lastSuccessfulRequestDetails); + streamAndShardId, lastSuccessfulRequestDetails); return; } if (!hasValidSubscriber()) { log.warn( "{}: (FanOutRecordsPublisher/Subscription#cancel) - Cancelled called even with an invalid subscriber. Last successful request details -- {}", - shardId, lastSuccessfulRequestDetails); + streamAndShardId, lastSuccessfulRequestDetails); } subscriber = null; if (flow != null) { log.debug( "{}: [SubscriptionLifetime]: (FanOutRecordsPublisher/Subscription#cancel) @ {} id: {}", - shardId, flow.connectionStartedAt, flow.subscribeToShardId); + streamAndShardId, flow.connectionStartedAt, flow.subscribeToShardId); flow.cancel(); availableQueueSpace = 0; } @@ -703,12 +715,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { public void onEventStream(SdkPublisher publisher) { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- Subscribe", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); if (!parent.isActiveFlow(this)) { this.isDisposed = true; log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- parent is disposed", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); parent.rejectSubscription(publisher); return; } @@ -716,7 +728,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { try { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- creating record subscription", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); subscription = new RecordSubscription(parent, this, connectionStartedAt, subscribeToShardId); publisher.subscribe(subscription); @@ -727,7 +739,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (Throwable t) { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#onEventStream) @ {} id: {} -- throwable during record subscription: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, t.getMessage()); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage()); parent.errorOccurred(this, t); } } @@ -736,7 +748,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { @Override public void responseReceived(SubscribeToShardResponse response) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#responseReceived) @ {} id: {} -- Response received. Request id - {}", - parent.shardId, connectionStartedAt, subscribeToShardId, response.responseMetadata().requestId()); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, response.responseMetadata().requestId()); final RequestDetails requestDetails = new RequestDetails(response.responseMetadata().requestId(), connectionStartedAt.toString()); parent.setLastSuccessfulRequestDetails(requestDetails); @@ -759,12 +771,12 @@ public class FanOutRecordsPublisher implements RecordsPublisher { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- {}: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); if (this.isDisposed) { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- This flow has been disposed, not dispatching error. {}: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); this.isErrorDispatched = true; } @@ -775,7 +787,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } else { log.debug( "{}: [SubscriptionLifetime]: (RecordFlow#exceptionOccurred) @ {} id: {} -- An error has previously been dispatched, not dispatching this error {}: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, throwable.getClass().getName(), throwable.getMessage()); } } @@ -802,7 +814,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (Exception e) { log.warn( "{}: Unable to enqueue the {} shutdown event due to capacity restrictions in delivery queue with remaining capacity {}. Ignoring. Last successful request details -- {}", - parent.shardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(), + parent.streamAndShardId, subscriptionShutdownEvent.getEventIdentifier(), parent.recordsDeliveryQueue.remainingCapacity(), parent.lastSuccessfulRequestDetails, subscriptionShutdownEvent.getShutdownEventThrowableOptional()); } } @@ -810,7 +822,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { private void executeComplete() { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Connection completed", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); if (isCancelled) { // @@ -820,13 +832,13 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // subscription, which was cancelled for a reason (usually queue overflow). // log.warn("{}: complete called on a cancelled subscription. Ignoring completion. Last successful request details -- {}", - parent.shardId, parent.lastSuccessfulRequestDetails); + parent.streamAndShardId, parent.lastSuccessfulRequestDetails); return; } if (this.isDisposed) { log.warn( "{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- This flow has been disposed not dispatching completion. Last successful request details -- {}", - parent.shardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.lastSuccessfulRequestDetails); return; } @@ -844,7 +856,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } catch (Throwable t) { log.error( "{}: [SubscriptionLifetime]: (RecordFlow#complete) @ {} id: {} -- Exception while trying to cancel failed subscription: {}", - parent.shardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getMessage(), t); } } } @@ -885,14 +897,14 @@ public class FanOutRecordsPublisher implements RecordsPublisher { public void cancel() { synchronized (parent.lockObject) { log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- Cancel called", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); flow.isCancelled = true; if (subscription != null) { subscription.cancel(); } else { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#cancel) @ {} id: {} -- SDK subscription is null", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } } } @@ -906,21 +918,21 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow.isCancelled) { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Subscription was cancelled before onSubscribe", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } if (flow.isDisposed) { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow has been disposed cancelling subscribe", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- RecordFlow requires cancelling", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); cancel(); } log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onSubscribe) @ {} id: {} -- Outstanding: {} items so requesting an item", - parent.shardId, connectionStartedAt, subscribeToShardId, parent.availableQueueSpace); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId, parent.availableQueueSpace); if (parent.availableQueueSpace > 0) { request(1); } @@ -933,7 +945,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { if (flow.shouldSubscriptionCancel()) { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onNext) @ {} id: {} -- RecordFlow requires cancelling", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); cancel(); return; } @@ -948,7 +960,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { @Override public void onError(Throwable t) { - log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#onError) @ {} id: {} -- {}: {}", parent.shardId, + log.debug("{}: [SubscriptionLifetime]: (RecordSubscription#onError) @ {} id: {} -- {}: {}", parent.streamAndShardId, connectionStartedAt, subscribeToShardId, t.getClass().getName(), t.getMessage()); // @@ -961,7 +973,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { public void onComplete() { log.debug( "{}: [SubscriptionLifetime]: (RecordSubscription#onComplete) @ {} id: {} -- Allowing RecordFlow to call onComplete", - parent.shardId, connectionStartedAt, subscribeToShardId); + parent.streamAndShardId, connectionStartedAt, subscribeToShardId); } } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcher.java index 1ea833a3..1605f941 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 @@ -63,7 +63,7 @@ public class KinesisDataFetcher { @NonNull private final KinesisAsyncClient kinesisClient; - @NonNull + @NonNull @Getter private final StreamIdentifier streamIdentifier; @NonNull private final String shardId; @@ -71,6 +71,7 @@ public class KinesisDataFetcher { @NonNull private final MetricsFactory metricsFactory; private final Duration maxFutureWait; + private final String streamAndShardId; @Deprecated public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory) { @@ -93,6 +94,7 @@ public class KinesisDataFetcher { this.maxRecords = maxRecords; this.metricsFactory = metricsFactory; this.maxFutureWait = maxFutureWait; + this.streamAndShardId = streamIdentifier.serialize() + ":" + shardId; } /** Note: This method has package level access for testing purposes. @@ -120,7 +122,7 @@ public class KinesisDataFetcher { try { return new AdvancingResult(getRecords(nextIterator)); } catch (ResourceNotFoundException e) { - log.info("Caught ResourceNotFoundException when fetching records for shard {}", shardId); + log.info("Caught ResourceNotFoundException when fetching records for shard {}", streamAndShardId); return TERMINAL_RESULT; } } else { @@ -182,14 +184,14 @@ public class KinesisDataFetcher { */ public void initialize(final String initialCheckpoint, final InitialPositionInStreamExtended initialPositionInStream) { - log.info("Initializing shard {} with {}", shardId, initialCheckpoint); + log.info("Initializing shard {} with {}", streamAndShardId, initialCheckpoint); advanceIteratorTo(initialCheckpoint, initialPositionInStream); isInitialized = true; } public void initialize(final ExtendedSequenceNumber initialCheckpoint, final InitialPositionInStreamExtended initialPositionInStream) { - log.info("Initializing shard {} with {}", shardId, initialCheckpoint.sequenceNumber()); + log.info("Initializing shard {} with {}", streamAndShardId, initialCheckpoint.sequenceNumber()); advanceIteratorTo(initialCheckpoint.sequenceNumber(), initialPositionInStream); isInitialized = true; } @@ -234,7 +236,7 @@ public class KinesisDataFetcher { throw new RetryableRetrievalException(e.getMessage(), e); } } catch (ResourceNotFoundException e) { - log.info("Caught ResourceNotFoundException when getting an iterator for shard {}", shardId, e); + log.info("Caught ResourceNotFoundException when getting an iterator for shard {}", streamAndShardId, e); nextIterator = null; } finally { MetricsUtil.addSuccessAndLatency(metricsScope, String.format("%s.%s", METRICS_PREFIX, "getShardIterator"), @@ -285,7 +287,7 @@ public class KinesisDataFetcher { throw exceptionManager.apply(e.getCause()); } catch (InterruptedException e) { // TODO: Check behavior - log.debug("Interrupt called on metod, shutdown initiated"); + log.debug("{} : Interrupt called on method, shutdown initiated", streamAndShardId); throw new RuntimeException(e); } catch (TimeoutException e) { throw new RetryableRetrievalException(e.getMessage(), e); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index dcd5e043..ba8aa117 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 @@ -91,7 +91,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { private final DefaultGetRecordsCacheDaemon defaultGetRecordsCacheDaemon; private boolean started = false; private final String operation; - private final String shardId; + private final String streamAndShardId; private Subscriber subscriber; @VisibleForTesting @Getter private final PublisherSession publisherSession; @@ -135,11 +135,11 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { // Handle records delivery ack and execute nextEventDispatchAction. // This method is not thread-safe and needs to be called after acquiring a monitor. - void handleRecordsDeliveryAck(RecordsDeliveryAck recordsDeliveryAck, String shardId, Runnable nextEventDispatchAction) { + void handleRecordsDeliveryAck(RecordsDeliveryAck recordsDeliveryAck, String streamAndShardId, Runnable nextEventDispatchAction) { final PrefetchRecordsRetrieved recordsToCheck = peekNextRecord(); // Verify if the ack matches the head of the queue and evict it. if (recordsToCheck != null && recordsToCheck.batchUniqueIdentifier().equals(recordsDeliveryAck.batchUniqueIdentifier())) { - evictPublishedRecordAndUpdateDemand(shardId); + evictPublishedRecordAndUpdateDemand(streamAndShardId); nextEventDispatchAction.run(); } else { // Log and ignore any other ack received. As long as an ack is received for head of the queue @@ -148,21 +148,21 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { final BatchUniqueIdentifier peekedBatchUniqueIdentifier = recordsToCheck == null ? null : recordsToCheck.batchUniqueIdentifier(); log.info("{} : Received a stale notification with id {} instead of expected id {} at {}. Will ignore.", - shardId, recordsDeliveryAck.batchUniqueIdentifier(), peekedBatchUniqueIdentifier, Instant.now()); + streamAndShardId, recordsDeliveryAck.batchUniqueIdentifier(), peekedBatchUniqueIdentifier, Instant.now()); } } // Evict the published record from the prefetch queue. // This method is not thread-safe and needs to be called after acquiring a monitor. @VisibleForTesting - RecordsRetrieved evictPublishedRecordAndUpdateDemand(String shardId) { + RecordsRetrieved evictPublishedRecordAndUpdateDemand(String streamAndShardId) { final PrefetchRecordsRetrieved result = prefetchRecordsQueue.poll(); if (result != null) { updateDemandTrackersOnPublish(result); } else { log.info( "{}: No record batch found while evicting from the prefetch queue. This indicates the prefetch buffer" - + "was reset.", shardId); + + "was reset.", streamAndShardId); } return result; } @@ -222,7 +222,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon(); Validate.notEmpty(operation, "Operation cannot be empty"); this.operation = operation; - this.shardId = shardId; + this.streamAndShardId = + this.getRecordsRetrievalStrategy.getDataFetcher().getStreamIdentifier().serialize() + ":" + shardId; } @Override @@ -234,7 +235,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended); if (!started) { - log.info("{} : Starting prefetching thread.", shardId); + log.info("{} : Starting prefetching thread.", streamAndShardId); executorService.execute(defaultGetRecordsCacheDaemon); } started = true; @@ -304,9 +305,9 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { @Override public synchronized void notify(RecordsDeliveryAck recordsDeliveryAck) { - publisherSession.handleRecordsDeliveryAck(recordsDeliveryAck, shardId, () -> drainQueueForRequests()); + publisherSession.handleRecordsDeliveryAck(recordsDeliveryAck, streamAndShardId, () -> drainQueueForRequests()); // Take action based on the time spent by the event in queue. - takeDelayedDeliveryActionIfRequired(shardId, lastEventDeliveryTime, log); + takeDelayedDeliveryActionIfRequired(streamAndShardId, lastEventDeliveryTime, log); } // Note : Do not make this method synchronous as notify() will not be able to evict any entry from the queue. @@ -403,7 +404,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { public void run() { while (!isShutdown) { if (Thread.currentThread().isInterrupted()) { - log.warn("{} : Prefetch thread was interrupted.", shardId); + log.warn("{} : Prefetch thread was interrupted.", streamAndShardId); break; } @@ -411,7 +412,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { try { makeRetrievalAttempt(); } catch(PositionResetException pre) { - log.debug("{} : Position was reset while attempting to add item to queue.", shardId); + log.debug("{} : Position was reset while attempting to add item to queue.", streamAndShardId); } finally { resetLock.readLock().unlock(); } @@ -447,23 +448,23 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { } catch (PositionResetException pse) { throw pse; } catch (RetryableRetrievalException rre) { - log.info("{} : Timeout occurred while waiting for response from Kinesis. Will retry the request.", shardId); + log.info("{} : Timeout occurred while waiting for response from Kinesis. Will retry the request.", streamAndShardId); } catch (InterruptedException e) { - log.info("{} : Thread was interrupted, indicating shutdown was called on the cache.", shardId); + log.info("{} : Thread was interrupted, indicating shutdown was called on the cache.", streamAndShardId); } catch (ExpiredIteratorException e) { log.info("{} : records threw ExpiredIteratorException - restarting" - + " after greatest seqNum passed to customer", shardId, e); + + " after greatest seqNum passed to customer", streamAndShardId, e); scope.addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.COUNT, MetricsLevel.SUMMARY); publisherSession.dataFetcher().restartIterator(); } catch (SdkException e) { - log.error("{} : Exception thrown while fetching records from Kinesis", shardId, e); + log.error("{} : Exception thrown while fetching records from Kinesis", streamAndShardId, e); } catch (Throwable e) { log.error("{} : Unexpected exception was thrown. This could probably be an issue or a bug." + " Please search for the exception/error online to check what is going on. If the " + "issue persists or is a recurring problem, feel free to open an issue on, " + - "https://github.com/awslabs/amazon-kinesis-client.", shardId, e); + "https://github.com/awslabs/amazon-kinesis-client.", streamAndShardId, e); } finally { MetricsUtil.endScope(scope); } @@ -475,7 +476,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { publisherSession.prefetchCounters().waitForConsumer(); } catch (InterruptedException ie) { log.info("{} : Thread was interrupted while waiting for the consumer. " + - "Shutdown has probably been started", shardId); + "Shutdown has probably been started", streamAndShardId); } } } @@ -522,14 +523,14 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { public synchronized void waitForConsumer() throws InterruptedException { if (!shouldGetNewRecords()) { - log.debug("{} : Queue is full waiting for consumer for {} ms", shardId, idleMillisBetweenCalls); + log.debug("{} : Queue is full waiting for consumer for {} ms", streamAndShardId, idleMillisBetweenCalls); this.wait(idleMillisBetweenCalls); } } public synchronized boolean shouldGetNewRecords() { if (log.isDebugEnabled()) { - log.debug("{} : Current Prefetch Counter States: {}", shardId, this.toString()); + log.debug("{} : Current Prefetch Counter States: {}", streamAndShardId, this.toString()); } return size < maxRecordsCount && byteSize < maxByteSize; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index 0cc50c2a..84851329 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 @@ -1156,6 +1156,45 @@ public class HierarchicalShardSyncerTest { // * Current leases: (4, 5, 7) // */ @Test + public void understandLeaseBehavior() { + final List shards = constructShardListForGraphA(); +// final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), +// newLease("shardId-7")); + + final List currentLeases = Collections.emptyList(); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + INITIAL_POSITION_LATEST); + + System.out.println("Leases : " + newLeases.stream().map(lease -> lease.leaseKey() + ":" + lease.checkpoint()).collect( + Collectors.joining())); + + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + + assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); + for (Lease lease : newLeases) { + assertThat("Unexpected lease: " + lease, expectedShardIdCheckpointMap.containsKey(lease.leaseKey()), + equalTo(true)); + assertThat(lease.checkpoint(), equalTo(expectedShardIdCheckpointMap.get(lease.leaseKey()))); + } + } + + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 7) + */ + @Test public void testDetermineNewLeasesToCreateSplitMergeLatest2() { final List shards = constructShardListForGraphA(); final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), From a6f767bf96b8c5ef4bd17d406efeb6c4a20778c9 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 23 Mar 2020 13:13:49 -0700 Subject: [PATCH 022/159] Shard Syncer and Logging changes --- .../amazon/kinesis/coordinator/Scheduler.java | 13 ++++++------- .../amazon/kinesis/lifecycle/ShutdownTask.java | 17 +++++++++-------- .../kinesis/coordinator/SchedulerTest.java | 3 +++ .../leases/HierarchicalShardSyncerTest.java | 3 ++- .../kinesis/lifecycle/ConsumerStatesTest.java | 4 ++-- .../polling/PrefetchRecordsPublisherTest.java | 3 ++- .../polling/RecordsFetcherFactoryTest.java | 6 ++++++ 7 files changed, 30 insertions(+), 19 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 c8354eff..945700a7 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 @@ -135,7 +135,7 @@ public class Scheduler implements Runnable { private final boolean isMultiStreamMode; // TODO : halo : make sure we generate streamConfig if entry not present. private final Map currentStreamConfigMap; - private final MultiStreamTracker multiStreamTracker; + private MultiStreamTracker multiStreamTracker; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; @@ -201,14 +201,13 @@ public class Scheduler implements Runnable { this.isMultiStreamMode = this.retrievalConfig.appStreamTracker().map( multiStreamTracker -> true, streamConfig -> false); this.currentStreamConfigMap = this.retrievalConfig.appStreamTracker().map( - multiStreamTracker -> - multiStreamTracker.streamConfigList().stream() - .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)), + multiStreamTracker -> { + this.multiStreamTracker = multiStreamTracker; + return multiStreamTracker.streamConfigList().stream() + .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); + }, streamConfig -> Collections.singletonMap(streamConfig.streamIdentifier(), streamConfig)); - this.multiStreamTracker = this.retrievalConfig.appStreamTracker().map( - multiStreamTracker -> multiStreamTracker, - streamConfig -> null); this.maxInitializationAttempts = this.coordinatorConfig.maxInitializationAttempts(); this.metricsFactory = this.metricsConfig.metricsFactory(); // Determine leaseSerializer based on availability of MultiStreamTracker. 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 81b954a6..2b4899e6 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 @@ -45,6 +45,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.function.Function; /** * Task for invoking the ShardRecordProcessor shutdown() callback. @@ -82,8 +83,8 @@ public class ShutdownTask implements ConsumerTask { private final TaskType taskType = TaskType.SHUTDOWN; - private String shardInfoId = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) - .orElse(shardInfo.shardId()); + private static final Function shardInfoIdProvider = shardInfo -> shardInfo + .streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()).orElse(shardInfo.shardId()); /* * Invokes ShardRecordProcessor shutdown() API. * (non-Javadoc) @@ -114,7 +115,7 @@ public class ShutdownTask implements ConsumerTask { if (CollectionUtils.isNullOrEmpty(latestShards) || !isShardInContextParentOfAny(latestShards)) { localReason = ShutdownReason.LEASE_LOST; dropLease(); - log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfoId); + log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfoIdProvider.apply(shardInfo)); } } @@ -126,7 +127,7 @@ public class ShutdownTask implements ConsumerTask { } log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", - shardInfoId, shardInfo.concurrencyToken(), localReason); + shardInfoIdProvider.apply(shardInfo), shardInfo.concurrencyToken(), localReason); final ShutdownInput shutdownInput = ShutdownInput.builder().shutdownReason(localReason) .checkpointer(recordProcessorCheckpointer).build(); final long startTime = System.currentTimeMillis(); @@ -137,7 +138,7 @@ public class ShutdownTask implements ConsumerTask { if (lastCheckpointValue == null || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfoId + ". Application must checkpoint upon shard end. " + + + shardInfoIdProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + "See ShardRecordProcessor.shardEnded javadocs for more information."); } } else { @@ -145,7 +146,7 @@ public class ShutdownTask implements ConsumerTask { } log.debug("Shutting down retrieval strategy."); recordsPublisher.shutdown(); - log.debug("Record processor completed shutdown() for shard {}", shardInfoId); + log.debug("Record processor completed shutdown() for shard {}", shardInfoIdProvider.apply(shardInfo)); } catch (Exception e) { applicationException = true; throw e; @@ -154,11 +155,11 @@ public class ShutdownTask implements ConsumerTask { } if (localReason == ShutdownReason.SHARD_END) { - log.debug("Looking for child shards of shard {}", shardInfoId); + log.debug("Looking for child shards of shard {}", shardInfoIdProvider.apply(shardInfo)); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); - log.debug("Finished checking for child shards of shard {}", shardInfoId); + log.debug("Finished checking for child shards of shard {}", shardInfoIdProvider.apply(shardInfo)); } return new TaskResult(null); 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 bd8c28e8..425af67f 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 @@ -57,6 +57,7 @@ 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; +import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; @@ -173,6 +174,7 @@ public class SchedulerTest { when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(MetricsFactory.class))).thenReturn(recordsPublisher); + when(shardDetector.streamIdentifier()).thenReturn(mock(StreamIdentifier.class)); scheduler = new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig); @@ -641,6 +643,7 @@ public class SchedulerTest { shardSyncTaskManagerMap.put(streamConfig.streamIdentifier(), shardSyncTaskManager); shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + when(shardDetector.streamIdentifier()).thenReturn(streamConfig.streamIdentifier()); if(shardSyncFirstAttemptFailure) { when(shardDetector.listShards()) .thenThrow(new RuntimeException("Service Exception")) 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 84851329..5008b912 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 @@ -98,6 +98,7 @@ public class HierarchicalShardSyncerTest { @Before public void setup() { hierarchicalShardSyncer = new HierarchicalShardSyncer(); + when(shardDetector.streamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("stream")); } private void setupMultiStream() { @@ -1155,7 +1156,7 @@ public class HierarchicalShardSyncerTest { // * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) // * Current leases: (4, 5, 7) // */ - @Test +// @Test public void understandLeaseBehavior() { final List shards = constructShardListForGraphA(); // final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), 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 23fb5dad..5d8e302f 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 @@ -119,10 +119,10 @@ public class ConsumerStatesTest { maxListShardsRetryAttempts, shouldCallProcessRecordsEvenForEmptyRecordList, idleTimeInMillis, INITIAL_POSITION_IN_STREAM, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, shardDetector, new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory); + when(shardInfo.shardId()).thenReturn("shardId-000000000000"); + when(shardInfo.streamIdentifierSerOpt()).thenReturn(Optional.of(StreamIdentifier.singleStreamInstance(STREAM_NAME).serialize())); consumer = spy(new ShardConsumer(recordsPublisher, executorService, shardInfo, logWarningForTaskAfterMillis, argument, taskExecutionListener, 0)); - - when(shardInfo.shardId()).thenReturn("shardId-000000000000"); when(recordProcessorCheckpointer.checkpointer()).thenReturn(checkpointer); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java index f5772aaf..a28ded63 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 @@ -76,6 +76,7 @@ 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.common.StreamIdentifier; import software.amazon.kinesis.lifecycle.ShardConsumerNotifyingSubscriber; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.metrics.NullMetricsFactory; @@ -120,7 +121,7 @@ public class PrefetchRecordsPublisherTest { @Before public void setup() { when(getRecordsRetrievalStrategy.getDataFetcher()).thenReturn(dataFetcher); - + when(dataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("testStream")); executorService = spy(Executors.newFixedThreadPool(1)); getRecordsCache = new PrefetchRecordsPublisher( MAX_SIZE, diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java index 81ad5b6d..d6d8b6d5 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.retrieval.polling; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.when; import org.junit.Before; import org.junit.Ignore; @@ -23,6 +24,7 @@ import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.DataFetchingStrategy; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @@ -37,11 +39,15 @@ public class RecordsFetcherFactoryTest { private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @Mock private MetricsFactory metricsFactory; + @Mock + private KinesisDataFetcher kinesisDataFetcher; @Before public void setUp() { MockitoAnnotations.initMocks(this); recordsFetcherFactory = new SimpleRecordsFetcherFactory(); + when(getRecordsRetrievalStrategy.getDataFetcher()).thenReturn(kinesisDataFetcher); + when(kinesisDataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("stream")); } @Test From a7ae4d3e24d936b7f408f6b10e68f5787e8e4076 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 23 Mar 2020 13:23:59 -0700 Subject: [PATCH 023/159] Minor refactoring --- .../fanout/FanOutRetrievalFactory.java | 8 +++- .../leases/HierarchicalShardSyncerTest.java | 39 ------------------- 2 files changed, 6 insertions(+), 41 deletions(-) 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 f609c1d9..719d2e54 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 @@ -53,10 +53,14 @@ public class FanOutRetrievalFactory implements RetrievalFactory { final String streamName; if(streamIdentifierStr.isPresent()) { streamName = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()).streamName(); + return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), + streamToConsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply), + streamIdentifierStr.get()); } else { streamName = defaultStreamName; + return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), + streamToConsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); } - return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), - streamToConsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); + } } 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 5008b912..9ca59edc 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 @@ -1146,45 +1146,6 @@ public class HierarchicalShardSyncerTest { } } -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) -// * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) -// * Current leases: (4, 5, 7) -// */ -// @Test - public void understandLeaseBehavior() { - final List shards = constructShardListForGraphA(); -// final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), -// newLease("shardId-7")); - - final List currentLeases = Collections.emptyList(); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); - - System.out.println("Leases : " + newLeases.stream().map(lease -> lease.leaseKey() + ":" + lease.checkpoint()).collect( - Collectors.joining())); - - final Map expectedShardIdCheckpointMap = new HashMap<>(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); - - assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); - for (Lease lease : newLeases) { - assertThat("Unexpected lease: " + lease, expectedShardIdCheckpointMap.containsKey(lease.leaseKey()), - equalTo(true)); - assertThat(lease.checkpoint(), equalTo(expectedShardIdCheckpointMap.get(lease.leaseKey()))); - } - } - - /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) * Shard structure (each level depicts a stream segment): From 35be75c347b241ae2f9a1bd5a34c3d476dda0686 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 23 Mar 2020 18:35:18 -0700 Subject: [PATCH 024/159] Addressing comments about unit tests --- .../amazon/kinesis/coordinator/Scheduler.java | 6 +++--- ...sticShuffleShardSyncLeaderDeciderTest.java | 19 +++++++++++++++---- 2 files changed, 18 insertions(+), 7 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 85234f7f..395882d9 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 @@ -363,14 +363,14 @@ public class Scheduler implements Runnable { long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); long waitUntil = System.currentTimeMillis() + waitTime; - boolean isLeaseTableEmpty = true; - while (System.currentTimeMillis() < waitUntil && (isLeaseTableEmpty = leaseRefresher.isLeaseTableEmpty())) { + boolean shouldInitiateLeaseSync = true; + while (System.currentTimeMillis() < waitUntil && (shouldInitiateLeaseSync = leaseRefresher.isLeaseTableEmpty())) { // check every 3 seconds if lease table is still empty, // to minimize contention between all workers bootstrapping at the same time log.info("Lease table is still empty. Checking again in {} ms", LEASE_TABLE_CHECK_FREQUENCY_MILLIS); Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS); } - return isLeaseTableEmpty; + return shouldInitiateLeaseSync; } private void waitUntilHashRangeCovered() throws InterruptedException { 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 b6ff3a0d..2e228a8a 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 @@ -34,6 +34,7 @@ import java.util.stream.Collectors; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.booleanThat; import static org.mockito.Mockito.when; import static software.amazon.kinesis.coordinator.DeterministicShuffleShardSyncLeaderDecider.DETERMINISTIC_SHUFFLE_SEED; @@ -72,10 +73,18 @@ public class DeterministicShuffleShardSyncLeaderDeciderTest { assertTrue("IsLeader should return true if no leases are returned", isLeader); } + @Test + public void testleaderElectionWithEmptyOwnerLeases() throws Exception { + List leases = getLeases(5, true, true, true); + when(leaseRefresher.listLeases()).thenReturn(leases); + boolean isLeader = leaderDecider.isLeader(WORKER_ID); + assertTrue("IsLeader should return false if leases have no owner", isLeader); + } + @Test public void testElectedLeadersAsPerExpectedShufflingOrder() throws Exception { - List leases = getLeases(5, 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) { @@ -92,7 +101,7 @@ public class DeterministicShuffleShardSyncLeaderDeciderTest { public void testElectedLeadersAsPerExpectedShufflingOrderWhenUniqueWorkersLessThanMaxLeaders() { this.numShardSyncWorkers = 5; // More than number of unique lease owners leaderDecider = new DeterministicShuffleShardSyncLeaderDecider(leaseRefresher, scheduledExecutorService, numShardSyncWorkers); - List leases = getLeases(3, false /* duplicateLeaseOwner */, true /* activeLeases */); + List leases = getLeases(3, false /*emptyLeaseOwner */, false /* duplicateLeaseOwner */, true /* activeLeases */); Set expectedLeaders = getExpectedLeaders(leases); // All lease owners should be present in expected leaders set, and they should all be leaders. for (Lease lease : leases) { @@ -101,14 +110,16 @@ public class DeterministicShuffleShardSyncLeaderDeciderTest { } } - private List getLeases(int count, boolean duplicateLeaseOwner, boolean activeLeases) { + private List getLeases(int count, boolean emptyLeaseOwner, boolean duplicateLeaseOwner, boolean activeLeases) { List leases = new ArrayList<>(); for (int i = 0; i < count; i++) { Lease lease = new Lease(); lease.leaseKey(LEASE_KEY + i); lease.checkpoint(activeLeases ? ExtendedSequenceNumber.LATEST : ExtendedSequenceNumber.SHARD_END); lease.leaseCounter(new Random().nextLong()); - lease.leaseOwner(LEASE_OWNER + (duplicateLeaseOwner ? "" : i)); + if (!emptyLeaseOwner) { + lease.leaseOwner(LEASE_OWNER + (duplicateLeaseOwner ? "" : i)); + } leases.add(lease); } return leases; From 98d2f946f4108153a38e2abfbed5cf1ad31ad9f2 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 24 Mar 2020 10:28:05 -0700 Subject: [PATCH 025/159] Fix test comment --- .../DeterministicShuffleShardSyncLeaderDeciderTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2e228a8a..b300f355 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 @@ -78,7 +78,7 @@ public class DeterministicShuffleShardSyncLeaderDeciderTest { List leases = getLeases(5, true, true, true); when(leaseRefresher.listLeases()).thenReturn(leases); boolean isLeader = leaderDecider.isLeader(WORKER_ID); - assertTrue("IsLeader should return false if leases have no owner", isLeader); + assertTrue("IsLeader should return true if leases have no owner", isLeader); } @Test From 391532da31f2f0c5e3dab6990a4f33f2b327af9b Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 25 Mar 2020 12:19:47 -0700 Subject: [PATCH 026/159] Adding comments for metric and Todo works --- .../java/software/amazon/kinesis/coordinator/Scheduler.java | 3 +++ .../java/software/amazon/kinesis/leases/ShardSyncTask.java | 3 +++ 2 files changed, 6 insertions(+) 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 395882d9..ff9d845e 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 @@ -375,6 +375,9 @@ public class Scheduler implements Runnable { private void waitUntilHashRangeCovered() throws InterruptedException { + // TODO: Currently this call is not in use. We may need to implement this method later. Created SIM to track the work: https://sim.amazon.com/issues/KinesisLTR-202 + // TODO: For future implementation, streamToShardSyncTaskManagerMap might not contain the most up to date snapshot of active streams. + // Should use currentStreamConfigMap to determine the streams to check. while (!leaderElectedPeriodicShardSyncManager.hashRangeCovered()) { // wait until entire hash range is covered log.info("Hash range is not covered yet. Checking again in {} ms", HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS); 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 b8f581c6..d7548a55 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 @@ -76,6 +76,9 @@ public class ShardSyncTask implements ConsumerTask { exception = e; shardSyncSuccess = false; } finally { + // NOTE: This metric is reflecting if a shard sync task succeeds. Customer can use this metric to monitor if + // their application encounter any shard sync failures. This metric can help to detect potential shard stuck issues + // that are due to shard sync failures. MetricsUtil.addSuccess(scope, "SyncShards", shardSyncSuccess, MetricsLevel.DETAILED); MetricsUtil.endScope(scope); } From e906a835f885a96a4a441f6ddeb99f247f790142 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 20 Mar 2020 05:52:27 -0700 Subject: [PATCH 027/159] Adding empty lease table sync. --- .../leases/HierarchicalShardSyncer.java | 365 ++++++++++++------ .../kinesis/leases/KinesisShardDetector.java | 13 +- .../amazon/kinesis/leases/ShardDetector.java | 3 + .../amazon/kinesis/leases/ShardSyncTask.java | 6 +- .../kinesis/leases/ShardSyncTaskManager.java | 2 + .../leases/exceptions/ShardSyncer.java | 2 +- .../kinesis/lifecycle/ShutdownTask.java | 2 +- .../leases/HierarchicalShardSyncerTest.java | 228 ++++++----- .../leases/ShardSyncTaskIntegrationTest.java | 3 +- .../kinesis/lifecycle/ShutdownTaskTest.java | 3 +- 10 files changed, 374 insertions(+), 253 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 f4143581..f4a1231c 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 @@ -24,12 +24,14 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; +import lombok.AllArgsConstructor; import lombok.Data; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; @@ -38,6 +40,8 @@ import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.awssdk.services.kinesis.model.ShardFilterType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStream; @@ -85,6 +89,7 @@ public class HierarchicalShardSyncer { * @param shardDetector * @param leaseRefresher * @param initialPosition + * @param garbageCollectLeases * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards * @param scope @@ -96,20 +101,24 @@ public class HierarchicalShardSyncer { // CHECKSTYLE:OFF CyclomaticComplexity public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, - final MetricsScope scope) throws DependencyException, InvalidStateException, - ProvisionedThroughputException, KinesisClientLibIOException { - final List latestShards = getShardList(shardDetector); - checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, cleanupLeasesOfCompletedShards, - ignoreUnexpectedChildShards, scope, latestShards); + final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, + final boolean ignoreUnexpectedChildShards, final MetricsScope scope) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + final List latestShards = leaseRefresher.isLeaseTableEmpty() ? + getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); + checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, garbageCollectLeases, + cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); } //Provide a pre-collcted list of shards to avoid calling ListShards API public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, - final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean cleanupLeasesOfCompletedShards, + final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, + final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards) - throws DependencyException, InvalidStateException, - ProvisionedThroughputException, KinesisClientLibIOException { + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + + final boolean isLeaseTableEmpty = leaseRefresher.isLeaseTableEmpty(); + if (!CollectionUtils.isNullOrEmpty(latestShards)) { log.debug("Num shards: {}", latestShards.size()); } @@ -125,8 +134,10 @@ public class HierarchicalShardSyncer { getLeasesForStream(shardDetector.streamIdentifier(), leaseRefresher) : leaseRefresher.listLeases(); final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamIdentifier()); - final List newLeasesToCreate = determineNewLeasesToCreate(latestShards, currentLeases, initialPosition, - inconsistentShardIds, multiStreamArgs); + final LeaseSynchronizer leaseSynchronizer = isLeaseTableEmpty ? new EmptyLeaseTableSynchronizer() : + new NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + final List newLeasesToCreate = determineNewLeasesToCreate(leaseSynchronizer, latestShards, currentLeases, + initialPosition, inconsistentShardIds, multiStreamArgs); log.debug("Num new leases to create: {}", newLeasesToCreate.size()); for (Lease lease : newLeasesToCreate) { long startTime = System.currentTimeMillis(); @@ -140,8 +151,10 @@ public class HierarchicalShardSyncer { } final List trackedLeases = new ArrayList<>(currentLeases); trackedLeases.addAll(newLeasesToCreate); - cleanupGarbageLeases(shardDetector, latestShards, trackedLeases, leaseRefresher, multiStreamArgs); - if (cleanupLeasesOfCompletedShards) { + if (!isLeaseTableEmpty && garbageCollectLeases) { + cleanupGarbageLeases(shardDetector, latestShards, trackedLeases, leaseRefresher, multiStreamArgs); + } + if (!isLeaseTableEmpty && cleanupLeasesOfCompletedShards) { cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases, leaseRefresher, multiStreamArgs); } @@ -299,6 +312,33 @@ public class HierarchicalShardSyncer { return shardIdToChildShardIdsMap; } + private static ShardFilter getShardFilterFromInitialPosition(InitialPositionInStreamExtended initialPositionInStreamExtended) { + + ShardFilter.Builder builder = ShardFilter.builder(); + + switch (initialPositionInStreamExtended.getInitialPositionInStream()) { + case LATEST: + builder = builder.type(ShardFilterType.AT_LATEST); + break; + case TRIM_HORIZON: + builder = builder.type(ShardFilterType.AT_TRIM_HORIZON); + break; + case AT_TIMESTAMP: + builder = builder.type(ShardFilterType.AT_TIMESTAMP).timestamp(initialPositionInStreamExtended.getTimestamp().toInstant()); + break; + } + return builder.build(); + } + + private static List getShardListAtInitialPosition(@NonNull final ShardDetector shardDetector, + InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException { + final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); + final Optional> shards = Optional.of(shardDetector.listShardsWithFilter(shardFilter)); + + return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream is not in ACTIVE OR UPDATING state - " + + "will retry getting the shard list.")); + } + private static List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { final List shards = shardDetector.listShards(); if (shards == null) { @@ -312,42 +352,7 @@ public class HierarchicalShardSyncer { * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. * - * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, - * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): - * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. - * If not, set checkpoint of the shard to the initial position specified by the client. - * To check if we need to create leases for ancestors, we use the following rules: - * * If we began (or will begin) processing data for a shard, then we must reach end of that shard before - * we begin processing data from any of its descendants. - * * A shard does not start processing data until data from all its parents has been processed. - * Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create - * leases corresponding to both the parents - the parent shard which is not a descendant will have - * its checkpoint set to Latest. - * - * We assume that if there is an existing lease for a shard, then either: - * * we have previously created a lease for its parent (if it was needed), or - * * the parent shard has expired. - * - * For example: - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | / \ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) - * New leases to create: (2, 6, 7, 8, 9, 10) - * - * The leases returned are sorted by the starting sequence number - following the same order - * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail - * before creating all the leases. - * - * If a shard has no existing lease, is open, and is a descendant of a parent which is still open, we ignore it - * here; this happens when the list of shards is inconsistent, which could be due to pagination delay for very - * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only - * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. - * - * + * @param leaseSynchronizer determines the strategy we'll be using to update any new leases. * @param shards List of all shards in Kinesis (we'll create new leases based on this set) * @param currentLeases List of current leases * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that @@ -355,81 +360,15 @@ public class HierarchicalShardSyncer { * @param inconsistentShardIds Set of child shard ids having open parents. * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ - static List determineNewLeasesToCreate(final List shards, final List currentLeases, - final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds, - final MultiStreamArgs multiStreamArgs) { - final Map shardIdToNewLeaseMap = new HashMap<>(); - final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); - - final Set shardIdsOfCurrentLeases = currentLeases.stream() - .peek(lease -> log.debug("Existing lease: {}", lease)) - .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) - .collect(Collectors.toSet()); - - final List openShards = getOpenShards(shards); - final Map memoizationContext = new HashMap<>(); - - // Iterate over the open shards and find those that don't have any lease entries. - for (Shard shard : openShards) { - final String shardId = shard.shardId(); - log.debug("Evaluating leases for open shard {} and its ancestors.", shardId); - if (shardIdsOfCurrentLeases.contains(shardId)) { - log.debug("Lease for shardId {} already exists. Not creating a lease", shardId); - } else if (inconsistentShardIds.contains(shardId)) { - log.info("shardId {} is an inconsistent child. Not creating a lease", shardId); - } else { - log.debug("Need to create a lease for shardId {}", shardId); - final Lease newLease = multiStreamArgs.isMultiStreamMode() ? - newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : - newKCLLease(shard); - final boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, - shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, - memoizationContext, multiStreamArgs); - - /** - * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the - * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a - * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side - * timestamp at or after the specified initial position timestamp. - * - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * - * Current leases: empty set - * - * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with - * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to - * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin - * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases - * would then be deleted since they won't have records with server-side timestamp at/after 206. And - * after that we will begin processing the descendant shards with epoch at/after 206 and we will - * return the records that meet the timestamp requirement for these shards. - */ - if (isDescendant - && !initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { - newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - newLease.checkpoint(convertToCheckpoint(initialPosition)); - } - log.debug("Set checkpoint of {} to {}", newLease.leaseKey(), newLease.checkpoint()); - shardIdToNewLeaseMap.put(shardId, newLease); - } - } - - final List newLeasesToCreate = new ArrayList<>(shardIdToNewLeaseMap.values()); - final Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMapOfAllKinesisShards, multiStreamArgs); - newLeasesToCreate.sort(startingSequenceNumberComparator); - return newLeasesToCreate; + static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, + final List currentLeases, final InitialPositionInStreamExtended initialPosition, + final Set inconsistentShardIds, final MultiStreamArgs multiStreamArgs) { + return leaseSynchronizer.determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds, multiStreamArgs); } - static List determineNewLeasesToCreate(final List shards, final List currentLeases, - final InitialPositionInStreamExtended initialPosition, final Set inconsistentShardIds) { - return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds, + static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, + final List currentLeases, final InitialPositionInStreamExtended initialPosition,final Set inconsistentShardIds) { + return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds, new MultiStreamArgs(false, null)); } @@ -437,10 +376,10 @@ public class HierarchicalShardSyncer { * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. */ - static List determineNewLeasesToCreate(final List shards, final List currentLeases, - final InitialPositionInStreamExtended initialPosition) { + static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, + final List currentLeases, final InitialPositionInStreamExtended initialPosition) { final Set inconsistentShardIds = new HashSet<>(); - return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds); + return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds); } /** @@ -450,6 +389,7 @@ public class HierarchicalShardSyncer { * See javadoc of determineNewLeasesToCreate() for rules and example. * * @param shardId The shardId to check. + * @param shardId The shardId to check. * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * location in the shard (when an application starts up for the first time - and there are no checkpoints). * @param shardIdsOfCurrentLeases The shardIds for the current leases. @@ -682,6 +622,7 @@ public class HierarchicalShardSyncer { if (!CollectionUtils.isNullOrEmpty(leasesOfClosedShards)) { assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, shardIdsOfClosedShards); + //TODO: Verify before LTR launch that ending sequence number is still returned from the service. Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( shardIdToShardMap, multiStreamArgs); leasesOfClosedShards.sort(startingSequenceNumberComparator); @@ -864,4 +805,178 @@ public class HierarchicalShardSyncer { private final StreamIdentifier streamIdentifier; } + @VisibleForTesting + static interface LeaseSynchronizer { + List determineNewLeasesToCreate(List shards, List currentLeases, + InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, + MultiStreamArgs multiStreamArgs); + } + + @Slf4j + @AllArgsConstructor + static class EmptyLeaseTableSynchronizer implements LeaseSynchronizer { + + @Override + public List determineNewLeasesToCreate(List shards, List currentLeases, + InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, MultiStreamArgs multiStreamArgs) { + final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); + + currentLeases.stream().peek(lease -> log.debug("Existing lease: {}", lease)) + .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .collect(Collectors.toSet()); + + final List newLeasesToCreate = getLeasesToCreateForOpenAndClosedShards(initialPosition, shards); + + //TODO: Verify before LTR launch that ending sequence number is still returned from the service. + final Comparator startingSequenceNumberComparator = + new StartingSequenceNumberAndShardIdBasedComparator(shardIdToShardMapOfAllKinesisShards, multiStreamArgs); + newLeasesToCreate.sort(startingSequenceNumberComparator); + return newLeasesToCreate; + } + + /** + * Helper method to create leases. For an empty lease table, we will be creating leases for all shards + * regardless of if they are open or closed. Closed shards will be unblocked via child shard information upon + * reaching SHARD_END. + */ + private List getLeasesToCreateForOpenAndClosedShards(InitialPositionInStreamExtended initialPosition, List shards) { + final Map shardIdToNewLeaseMap = new HashMap<>(); + + for (Shard shard : shards) { + final String shardId = shard.shardId(); + final Lease lease = newKCLLease(shard); + lease.checkpoint(convertToCheckpoint(initialPosition)); + + log.debug("Need to create a lease for shard with shardId {}", shardId); + + shardIdToNewLeaseMap.put(shardId, lease); + } + + return new ArrayList(shardIdToNewLeaseMap.values()); + } + } + + + @Slf4j + @AllArgsConstructor + static class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { + + private final ShardDetector shardDetector; + private final Map shardIdToShardMap; + private final Map> shardIdToChildShardIdsMap; + + /** + * Determine new leases to create and their initial checkpoint. + * Note: Package level access only for testing purposes. + *

+ * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, + * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): + * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. + * If not, set checkpoint of the shard to the initial position specified by the client. + * To check if we need to create leases for ancestors, we use the following rules: + * * If we began (or will begin) processing data for a shard, then we must reach end of that shard before + * we begin processing data from any of its descendants. + * * A shard does not start processing data until data from all its parents has been processed. + * Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create + * leases corresponding to both the parents - the parent shard which is not a descendant will have + * its checkpoint set to Latest. + *

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

+ * For example: + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | / \ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (3, 4, 5) + * New leases to create: (2, 6, 7, 8, 9, 10) + *

+ * The leases returned are sorted by the starting sequence number - following the same order + * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail + * before creating all the leases. + *

+ * If a shard has no existing lease, is open, and is a descendant of a parent which is still open, we ignore it + * here; this happens when the list of shards is inconsistent, which could be due to pagination delay for very + * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only + * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. + * + * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard + */ + @Override + public synchronized List determineNewLeasesToCreate(List shards, List currentLeases, + InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, MultiStreamArgs multiStreamArgs) { + final Map shardIdToNewLeaseMap = new HashMap<>(); + final Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); + + final Set shardIdsOfCurrentLeases = currentLeases.stream() + .peek(lease -> log.debug("Existing lease: {}", lease)) + .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) + .collect(Collectors.toSet()); + + final List openShards = getOpenShards(shards); + final Map memoizationContext = new HashMap<>(); + + // Iterate over the open shards and find those that don't have any lease entries. + for (Shard shard : openShards) { + final String shardId = shard.shardId(); + log.debug("Evaluating leases for open shard {} and its ancestors.", shardId); + if (shardIdsOfCurrentLeases.contains(shardId)) { + log.debug("Lease for shardId {} already exists. Not creating a lease", shardId); + } else if (inconsistentShardIds.contains(shardId)) { + log.info("shardId {} is an inconsistent child. Not creating a lease", shardId); + } else { + log.debug("Need to create a lease for shardId {}", shardId); + final Lease newLease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : + newKCLLease(shard); + final boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, + shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, + memoizationContext, multiStreamArgs); + + /** + * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the + * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a + * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side + * timestamp at or after the specified initial position timestamp. + * + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: empty set + * + * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with + * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to + * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin + * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases + * would then be deleted since they won't have records with server-side timestamp at/after 206. And + * after that we will begin processing the descendant shards with epoch at/after 206 and we will + * return the records that meet the timestamp requirement for these shards. + */ + if (isDescendant + && !initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { + newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } else { + newLease.checkpoint(convertToCheckpoint(initialPosition)); + } + log.debug("Set checkpoint of {} to {}", newLease.leaseKey(), newLease.checkpoint()); + shardIdToNewLeaseMap.put(shardId, newLease); + } + } + + final List newLeasesToCreate = new ArrayList<>(shardIdToNewLeaseMap.values()); + //TODO: Verify before LTR launch that ending sequence number is still returned from the service. + final Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( + shardIdToShardMapOfAllKinesisShards, multiStreamArgs); + newLeasesToCreate.sort(startingSequenceNumberComparator); + return newLeasesToCreate; + } + } } 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 0c495558..c0c3bdee 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 @@ -42,6 +42,7 @@ import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; @@ -149,12 +150,18 @@ public class KinesisShardDetector implements ShardDetector { @Override @Synchronized public List listShards() { + return listShardsWithFilter(null); + } + + @Override + @Synchronized + public List listShardsWithFilter(ShardFilter shardFilter) { final List shards = new ArrayList<>(); ListShardsResponse result; String nextToken = null; do { - result = listShards(nextToken); + result = listShards(shardFilter, nextToken); if (result == null) { /* @@ -172,13 +179,13 @@ public class KinesisShardDetector implements ShardDetector { return shards; } - private ListShardsResponse listShards(final String nextToken) { + private ListShardsResponse listShards(ShardFilter shardFilter, final String nextToken) { final AWSExceptionManager exceptionManager = new AWSExceptionManager(); exceptionManager.add(LimitExceededException.class, t -> t); exceptionManager.add(ResourceInUseException.class, t -> t); exceptionManager.add(KinesisException.class, t -> t); - ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder(); + ListShardsRequest.Builder request = KinesisRequestsBuilder.listShardsRequestBuilder().shardFilter(shardFilter); if (StringUtils.isEmpty(nextToken)) { request = request.streamName(streamIdentifier.streamName()); } else { 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 6ae012e6..1b2822ee 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 @@ -16,6 +16,7 @@ package software.amazon.kinesis.leases; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.kinesis.common.StreamIdentifier; import java.util.List; @@ -28,6 +29,8 @@ public interface ShardDetector { List listShards(); + List listShardsWithFilter(ShardFilter shardFilter); + default StreamIdentifier streamIdentifier() { throw new UnsupportedOperationException("StreamName not available"); } 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 d7548a55..9291abf4 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 @@ -46,6 +46,7 @@ public class ShardSyncTask implements ConsumerTask { @NonNull private final InitialPositionInStreamExtended initialPosition; private final boolean cleanupLeasesUponShardCompletion; + private final boolean garbageCollectLeases; private final boolean ignoreUnexpectedChildShards; private final long shardSyncTaskIdleTimeMillis; @NonNull @@ -66,8 +67,9 @@ public class ShardSyncTask implements ConsumerTask { boolean shardSyncSuccess = true; try { - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, scope); + hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, + initialPosition, cleanupLeasesUponShardCompletion, garbageCollectLeases, ignoreUnexpectedChildShards, scope); + if (shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); } 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 8c8e0464..3933b70a 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 @@ -128,6 +128,7 @@ public class ShardSyncTaskManager { leaseRefresher, initialPositionInStream, cleanupLeasesUponShardCompletion, + true, ignoreUnexpectedChildShards, shardSyncIdleTimeMillis, hierarchicalShardSyncer, @@ -166,6 +167,7 @@ public class ShardSyncTaskManager { leaseRefresher, initialPositionInStream, cleanupLeasesUponShardCompletion, + true, ignoreUnexpectedChildShards, shardSyncIdleTimeMillis, hierarchicalShardSyncer, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java index 4e9245f6..a14ec784 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 @@ -41,6 +41,6 @@ public class ShardSyncer { final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope); + true, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 2bfcd358..9f8ee1d1 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 @@ -155,7 +155,7 @@ public class ShutdownTask implements ConsumerTask { log.debug("Looking for child shards of shard {}", shardInfo.shardId()); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); + initialPositionInStream, cleanupLeasesOfCompletedShards, true, ignoreUnexpectedChildShards, scope, latestShards); log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); } 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 0cc50c2a..0e0c0d29 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 @@ -80,6 +80,7 @@ public class HierarchicalShardSyncerTest { private static final HierarchicalShardSyncer.MultiStreamArgs MULTI_STREAM_ARGS = new HierarchicalShardSyncer.MultiStreamArgs( MULTISTREAM_MODE_ON, StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); + private final boolean garbageCollectLeases = true; private final boolean cleanupLeasesOfCompletedShards = true; private final boolean ignoreUnexpectedChildShards = false; @@ -112,9 +113,10 @@ public class HierarchicalShardSyncerTest { public void testDetermineNewLeasesToCreateNoShards() { final List shards = Collections.emptyList(); final List leases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - assertThat(HierarchicalShardSyncer.determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST).isEmpty(), - equalTo(true)); + assertThat(HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, + INITIAL_POSITION_LATEST).isEmpty(), equalTo(true)); } /** @@ -123,10 +125,11 @@ public class HierarchicalShardSyncerTest { @Test public void testDetermineNewLeasesToCreateNoShardsForMultiStream() { final List shards = Collections.emptyList(); final List leases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); assertThat(HierarchicalShardSyncer - .determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS) - .isEmpty(), equalTo(true)); + .determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, leases, INITIAL_POSITION_LATEST, + new HashSet<>(), MULTI_STREAM_ARGS).isEmpty(), equalTo(true)); } /** @@ -141,9 +144,10 @@ public class HierarchicalShardSyncerTest { final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); final List currentLeases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_LATEST); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); @@ -163,9 +167,10 @@ public class HierarchicalShardSyncerTest { final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); final List currentLeases = Collections.emptyList(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS); + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); @@ -174,59 +179,6 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseKeys, equalTo(expectedLeaseIds)); } - /** - * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but - * one of the shards was marked as inconsistent. - */ - @Test - public void testDetermineNewLeasesToCreate0Leases0Reshards1Inconsistent() { - final String shardId0 = "shardId-0"; - final String shardId1 = "shardId-1"; - final String shardId2 = "shardId-2"; - final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), - ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), - ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); - final List currentLeases = Collections.emptyList(); - - final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST, inconsistentShardIds); - final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); - assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); - assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); - } - - /** - * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but - * one of the shards was marked as inconsistent. - */ - @Test - public void testDetermineNewLeasesToCreate0Leases0Reshards1InconsistentMultiStream() { - final String shardId0 = "shardId-0"; - final String shardId1 = "shardId-1"; - final String shardId2 = "shardId-2"; - final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), - ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), - ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); - final List currentLeases = Collections.emptyList(); - - final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); - final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set expectedLeaseShardIds = new HashSet<>( - toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); - assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); - assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); - } - /** * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) */ @@ -258,7 +210,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -292,7 +244,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -334,7 +286,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -370,7 +322,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -406,7 +358,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, false, SCOPE, new ArrayList()); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, new ArrayList()); final Set expectedShardIds = new HashSet<>(); @@ -446,7 +398,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, false, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -466,7 +418,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, false, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -501,7 +453,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, true, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -546,7 +498,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - cleanupLeasesOfCompletedShards, true, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -602,7 +554,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -617,7 +569,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases present, with shardId-0 being at ShardEnd causing cleanup. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) @@ -677,7 +629,7 @@ public class HierarchicalShardSyncerTest { // Initial call: Call to create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -694,7 +646,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete fails. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); } finally { List deleteLeases = leaseDeleteCaptor.getAllValues(); Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -718,7 +670,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete passes. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); deleteLeases = leaseDeleteCaptor.getAllValues(); @@ -779,7 +731,7 @@ public class HierarchicalShardSyncerTest { // Initial call: Call to create leases. Fails on ListLeases hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -789,7 +741,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases not present, leases will be created. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); @@ -804,7 +756,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases present, belongs to TestOwner, shardId-0 is at ShardEnd should be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -870,7 +822,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. Create lease Fails hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -879,7 +831,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); @@ -894,7 +846,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases are present, shardId-0 is at ShardEnd needs to be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -984,7 +936,7 @@ public class HierarchicalShardSyncerTest { doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); @@ -1013,7 +965,7 @@ public class HierarchicalShardSyncerTest { doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); setupMultiStream(); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); @@ -1045,7 +997,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - cleanupLeasesOfCompletedShards, false, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -1069,6 +1021,7 @@ public class HierarchicalShardSyncerTest { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; final List currentLeases = new ArrayList<>(); + final HierarchicalShardSyncer.LeaseSynchronizer emptyLeaseTableSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), @@ -1080,7 +1033,7 @@ public class HierarchicalShardSyncerTest { final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); for (InitialPositionInStreamExtended initialPosition : initialPositions) { - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, initialPosition); assertThat(newLeases.size(), equalTo(2)); @@ -1092,24 +1045,6 @@ public class HierarchicalShardSyncerTest { } } - @Test - public void testDetermineNewLeasesToCreateIgnoreClosedShard() { - final String lastShardId = "shardId-1"; - final List currentLeases = new ArrayList<>(); - - final List shards = Arrays.asList( - ShardObjectHelper.newShard("shardId-0", null, null, - ShardObjectHelper.newSequenceNumberRange("303", "404")), - ShardObjectHelper.newShard(lastShardId, null, null, - ShardObjectHelper.newSequenceNumberRange("405", null))); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); - - assertThat(newLeases.size(), equalTo(1)); - assertThat(newLeases.get(0).leaseKey(), equalTo(lastShardId)); - } - // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) // * Shard structure (each level depicts a stream segment): @@ -1125,9 +1060,15 @@ public class HierarchicalShardSyncerTest { final List shards = constructShardListForGraphA(); final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), newLease("shardId-5")); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_LATEST); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); @@ -1161,8 +1102,15 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), newLease("shardId-7")); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_LATEST); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_LATEST); final Map expectedShardIdCheckpointMap = new HashMap<>(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); @@ -1194,8 +1142,15 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), newLease("shardId-5")); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_TRIM_HORIZON); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final List checkpoints = newLeases.stream().map(Lease::checkpoint) @@ -1229,8 +1184,15 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), newLease("shardId-7")); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_TRIM_HORIZON); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final List checkpoints = newLeases.stream().map(Lease::checkpoint) @@ -1259,8 +1221,15 @@ public class HierarchicalShardSyncerTest { final List shards = constructShardListForGraphB(); final List currentLeases = new ArrayList<>(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_TRIM_HORIZON); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final List checkpoints = newLeases.stream().map(Lease::checkpoint) @@ -1294,8 +1263,15 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), newLease("shardId-5")); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_AT_TIMESTAMP); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final List checkpoints = newLeases.stream().map(Lease::checkpoint) .collect(Collectors.toList()); @@ -1328,8 +1304,15 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), newLease("shardId-7")); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_AT_TIMESTAMP); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final List checkpoints = newLeases.stream().map(Lease::checkpoint) .collect(Collectors.toList()); @@ -1355,8 +1338,15 @@ public class HierarchicalShardSyncerTest { final List shards = constructShardListForGraphB(); final List currentLeases = new ArrayList<>(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, - INITIAL_POSITION_AT_TIMESTAMP); + final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer + .constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, + shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final List checkpoints = newLeases.stream().map(Lease::checkpoint) .collect(Collectors.toList()); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java index 42b826d1..ce6ce386 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardSyncTaskIntegrationTest.java @@ -118,7 +118,8 @@ public class ShardSyncTaskIntegrationTest { leaseRefresher.deleteAll(); Set shardIds = shardDetector.listShards().stream().map(Shard::shardId).collect(Collectors.toSet()); ShardSyncTask syncTask = new ShardSyncTask(shardDetector, leaseRefresher, - InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), false, false, 0L, + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), + false, true, false, 0L, hierarchicalShardSyncer, NULL_METRICS_FACTORY); syncTask.call(); List leases = leaseRefresher.listLeases(); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java index 6af62edb..44b2991b 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,6 +18,7 @@ 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.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; @@ -136,7 +137,7 @@ public class ShutdownTaskTest { throw new KinesisClientLibIOException("KinesisClientLibIOException"); }).when(hierarchicalShardSyncer) .checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, INITIAL_POSITION_TRIM_HORIZON, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, + true, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, NULL_METRICS_FACTORY.createMetrics(), latestShards); final TaskResult result = task.call(); From 1ccbe614eb6005262e81845ec774869b64215472 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 20 Mar 2020 06:00:26 -0700 Subject: [PATCH 028/159] Re-adding multistream tests. --- .../leases/HierarchicalShardSyncerTest.java | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) 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 0e0c0d29..2b5d562b 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 @@ -179,6 +179,59 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseKeys, equalTo(expectedLeaseIds)); } + /** + * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but + * one of the shards was marked as inconsistent. + */ + @Test + public void testDetermineNewLeasesToCreate0Leases0Reshards1Inconsistent() { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + final String shardId2 = "shardId-2"; + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); + final List currentLeases = Collections.emptyList(); + + final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + INITIAL_POSITION_LATEST, inconsistentShardIds); + final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); + assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + } + + /** + * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but + * one of the shards was marked as inconsistent. + */ + @Test + public void testDetermineNewLeasesToCreate0Leases0Reshards1InconsistentMultiStream() { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + final String shardId2 = "shardId-2"; + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), + ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); + final List currentLeases = Collections.emptyList(); + + final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); + final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + final Set expectedLeaseShardIds = new HashSet<>( + toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); + assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); + } + /** * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) */ @@ -1045,6 +1098,24 @@ public class HierarchicalShardSyncerTest { } } + @Test + public void testDetermineNewLeasesToCreateIgnoreClosedShard() { + final String lastShardId = "shardId-1"; + final List currentLeases = new ArrayList<>(); + + final List shards = Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, + ShardObjectHelper.newSequenceNumberRange("303", "404")), + ShardObjectHelper.newShard(lastShardId, null, null, + ShardObjectHelper.newSequenceNumberRange("405", null))); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + INITIAL_POSITION_LATEST); + + assertThat(newLeases.size(), equalTo(1)); + assertThat(newLeases.get(0).leaseKey(), equalTo(lastShardId)); + } + // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) // * Shard structure (each level depicts a stream segment): From 0052b5799c2de48ecbf04b7e0ba259b3e2bae73d Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 20 Mar 2020 13:12:19 -0700 Subject: [PATCH 029/159] Multistream support for leases. --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 f4a1231c..c093dced 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 @@ -839,12 +839,14 @@ public class HierarchicalShardSyncer { * regardless of if they are open or closed. Closed shards will be unblocked via child shard information upon * reaching SHARD_END. */ - private List getLeasesToCreateForOpenAndClosedShards(InitialPositionInStreamExtended initialPosition, List shards) { + private List getLeasesToCreateForOpenAndClosedShards(InitialPositionInStreamExtended initialPosition, + List shards, MultiStreamArgs multiStreamArgs) { final Map shardIdToNewLeaseMap = new HashMap<>(); for (Shard shard : shards) { final String shardId = shard.shardId(); - final Lease lease = newKCLLease(shard); + final Lease lease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier) : newKCLLease(shard); lease.checkpoint(convertToCheckpoint(initialPosition)); log.debug("Need to create a lease for shard with shardId {}", shardId); From 30ef7f62b1d1385f713a8a0977eabbd43c7a2c86 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 20 Mar 2020 13:12:32 -0700 Subject: [PATCH 030/159] TODO to add multistream support to read from lease tables. --- .../software/amazon/kinesis/leases/HierarchicalShardSyncer.java | 1 + 1 file changed, 1 insertion(+) 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 c093dced..ce658c51 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 @@ -117,6 +117,7 @@ public class HierarchicalShardSyncer { final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 final boolean isLeaseTableEmpty = leaseRefresher.isLeaseTableEmpty(); if (!CollectionUtils.isNullOrEmpty(latestShards)) { From 892218d8b5a92c9727032821e2f85e262cae96d7 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 23 Mar 2020 13:27:03 -0400 Subject: [PATCH 031/159] Adding multistreaming changes --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 2 +- .../kinesis/leases/HierarchicalShardSyncerTest.java | 10 +++++++--- 2 files changed, 8 insertions(+), 4 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 ce658c51..7829a2de 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 @@ -826,7 +826,7 @@ public class HierarchicalShardSyncer { .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)) .collect(Collectors.toSet()); - final List newLeasesToCreate = getLeasesToCreateForOpenAndClosedShards(initialPosition, shards); + final List newLeasesToCreate = getLeasesToCreateForOpenAndClosedShards(initialPosition, shards, multiStreamArgs); //TODO: Verify before LTR launch that ending sequence number is still returned from the service. final Comparator startingSequenceNumberComparator = 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 2b5d562b..b0fd5c6d 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 @@ -196,8 +196,9 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Collections.emptyList(); final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); @@ -222,8 +223,9 @@ public class HierarchicalShardSyncerTest { final List currentLeases = Collections.emptyList(); final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseShardIds = new HashSet<>( @@ -1109,7 +1111,9 @@ public class HierarchicalShardSyncerTest { ShardObjectHelper.newShard(lastShardId, null, null, ShardObjectHelper.newSequenceNumberRange("405", null))); - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(shards, currentLeases, + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + + final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); assertThat(newLeases.size(), equalTo(1)); From 039dd176f8ad9cfae24a4eaa355f6b9c17046c4c Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 24 Mar 2020 04:35:00 -0400 Subject: [PATCH 032/159] Fix existing unit tests. --- .../kinesis/lifecycle/ShutdownTask.java | 2 +- .../leases/HierarchicalShardSyncerTest.java | 48 +++++++++++++++---- .../kinesis/lifecycle/ShutdownTaskTest.java | 2 +- 3 files changed, 40 insertions(+), 12 deletions(-) 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 9f8ee1d1..55687a1e 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 @@ -155,7 +155,7 @@ public class ShutdownTask implements ConsumerTask { log.debug("Looking for child shards of shard {}", shardInfo.shardId()); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, cleanupLeasesOfCompletedShards, true, ignoreUnexpectedChildShards, scope, latestShards); + initialPositionInStream, false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); } 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 b0fd5c6d..596ef545 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 @@ -41,6 +41,7 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.junit.Before; @@ -180,7 +181,7 @@ public class HierarchicalShardSyncerTest { } /** - * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but + * Test determineNewLeasesToCreate() where there is one lease and no resharding operations have been performed, but * one of the shards was marked as inconsistent. */ @Test @@ -188,15 +189,22 @@ public class HierarchicalShardSyncerTest { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; final String shardId2 = "shardId-2"; + final String shardId3 = "shardId-3"; final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId3, null, null, sequenceRange)); + final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); - final List currentLeases = Collections.emptyList(); + final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); + final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); - final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(); + + Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); @@ -215,15 +223,22 @@ public class HierarchicalShardSyncerTest { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; final String shardId2 = "shardId-2"; + final String shardId3 = "shardId-3"; final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId3, null, null, sequenceRange)); + final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), ShardObjectHelper.newShard(shardId1, null, null, sequenceRange), ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); - final List currentLeases = Collections.emptyList(); + final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); + final List currentLeases = new ArrayList(createMultiStreamLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo")); final Set inconsistentShardIds = new HashSet<>(Collections.singletonList(shardId2)); - final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + + Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); @@ -1100,18 +1115,31 @@ public class HierarchicalShardSyncerTest { } } + /** + * Tests that leases are not created for closed shards. + */ @Test public void testDetermineNewLeasesToCreateIgnoreClosedShard() { final String lastShardId = "shardId-1"; - final List currentLeases = new ArrayList<>(); - final List shards = Arrays.asList( + final List shardsWithoutLeases = Arrays.asList( ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("303", "404")), ShardObjectHelper.newShard(lastShardId, null, null, ShardObjectHelper.newSequenceNumberRange("405", null))); - final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = new HierarchicalShardSyncer.EmptyLeaseTableSynchronizer(); + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard("shardId-2", null, + null, ShardObjectHelper.newSequenceNumberRange("202", "302"))); + + final List shards = Stream.of(shardsWithLeases, shardsWithoutLeases).flatMap(x -> x.stream()).collect(Collectors.toList()); + final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); + final Set inconsistentShardIds = Collections.emptySet(); + + Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); + Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final HierarchicalShardSyncer.LeaseSynchronizer leaseSynchronizer = + new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); 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 44b2991b..927fbf4f 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 @@ -137,7 +137,7 @@ public class ShutdownTaskTest { throw new KinesisClientLibIOException("KinesisClientLibIOException"); }).when(hierarchicalShardSyncer) .checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, INITIAL_POSITION_TRIM_HORIZON, - true, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, + false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, NULL_METRICS_FACTORY.createMetrics(), latestShards); final TaskResult result = task.call(); From 16cf142b44ae370a611bee109b8ad36bf4f14f26 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 24 Mar 2020 06:36:36 -0400 Subject: [PATCH 033/159] Adding unit tests for bootstrapping cases. --- .../leases/HierarchicalShardSyncerTest.java | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) 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 596ef545..cf393d92 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -21,6 +21,7 @@ package software.amazon.kinesis.leases; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; import static org.mockito.Matchers.any; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; @@ -29,6 +30,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.math.BigInteger; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -54,6 +56,8 @@ import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.model.HashKeyRange; import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; import software.amazon.awssdk.services.kinesis.model.Shard; +import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.awssdk.services.kinesis.model.ShardFilterType; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; @@ -1608,6 +1612,62 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseMap.isEmpty(), equalTo(true)); } + @Test + public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtLatest() throws Exception { + ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_LATEST).build(); + testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_LATEST, shardFilter); + } + + @Test + public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTrimHorizon() throws Exception { + ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build(); + testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_TRIM_HORIZON, shardFilter); + } + + @Test + public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTimestamp() throws Exception { + ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_TIMESTAMP).timestamp(new Date(1000L).toInstant()).build(); + testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_AT_TIMESTAMP, shardFilter); + } + + public void testEmptyLeaseTableBootstrapUsesListShardsWithFilter(InitialPositionInStreamExtended initialPosition, ShardFilter shardFilter) throws Exception { + final String shardId0 = "shardId-0"; + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, ShardObjectHelper.newSequenceNumberRange("1", null))); + final List currentLeases = Collections.emptyList(); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShards()).thenReturn(shards); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(currentLeases); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + + verify(shardDetector, atLeast(1)).listShardsWithFilter(shardFilter); + verify(shardDetector, never()).listShards(); + } + + @Test + public void testNonEmptyLeaseTableUsesListShards() throws Exception { + final String shardId0 = "shardId-0"; + final String shardId1 = "shardId-1"; + + final List shardsWithLeases = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"))); + final List shardsWithoutLeases = Arrays.asList(ShardObjectHelper.newShard(shardId1, null, null, ShardObjectHelper.newSequenceNumberRange("3", "4"))); + + final List currentLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, "foo"); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(false); + when(shardDetector.listShards()).thenReturn(shardsWithoutLeases); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(currentLeases); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + + verify(shardDetector, atLeast(1)).listShards(); + } + // /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors - two parents, there is a lease for one parent. // */ From 8609820f203df91c4ab73ef26f748899f3da5aa7 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Thu, 26 Mar 2020 05:16:32 -0400 Subject: [PATCH 034/159] Adding java docs --- .../leases/HierarchicalShardSyncer.java | 35 +++++++++++++++++++ .../leases/HierarchicalShardSyncerTest.java | 14 +++++++- 2 files changed, 48 insertions(+), 1 deletion(-) 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 7829a2de..d384bfe5 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 @@ -313,6 +313,11 @@ public class HierarchicalShardSyncer { return shardIdToChildShardIdsMap; } + /** + * Helper method to resolve the correct shard filter to use when listing shards from a position in a stream. + * @param initialPositionInStreamExtended + * @return + */ private static ShardFilter getShardFilterFromInitialPosition(InitialPositionInStreamExtended initialPositionInStreamExtended) { ShardFilter.Builder builder = ShardFilter.builder(); @@ -806,17 +811,44 @@ public class HierarchicalShardSyncer { private final StreamIdentifier streamIdentifier; } + /** + * Interface to determine how to create new leases. + */ @VisibleForTesting static interface LeaseSynchronizer { + /** + * Determines how to create leases. + * @param shards + * @param currentLeases + * @param initialPosition + * @param inconsistentShardIds + * @param multiStreamArgs + * @return + */ List determineNewLeasesToCreate(List shards, List currentLeases, InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, MultiStreamArgs multiStreamArgs); } + /** + * Class to help create leases when the table is initially empty. + */ @Slf4j @AllArgsConstructor static class EmptyLeaseTableSynchronizer implements LeaseSynchronizer { + /** + * Determines how to create leases when the lease table is initially empty. For this, we read all shards where + * the KCL is reading from. For any shards which are closed, we will discover their child shards through GetRecords + * child shard information. + * + * @param shards + * @param currentLeases + * @param initialPosition + * @param inconsistentShardIds + * @param multiStreamArgs + * @return + */ @Override public List determineNewLeasesToCreate(List shards, List currentLeases, InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds, MultiStreamArgs multiStreamArgs) { @@ -860,6 +892,9 @@ public class HierarchicalShardSyncer { } + /** + * Class to help create leases when the lease table is not initially empty. + */ @Slf4j @AllArgsConstructor static class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { 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 cf393d92..d8684cd7 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 @@ -1612,18 +1612,30 @@ public class HierarchicalShardSyncerTest { assertThat(newLeaseMap.isEmpty(), equalTo(true)); } + /** + * Tests that when reading from TIP, we use the AT_LATEST shard filter. + * @throws Exception + */ @Test public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtLatest() throws Exception { ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_LATEST).build(); testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_LATEST, shardFilter); } + /** + * Tests that when reading from TRIM, we use the TRIM_HORIZON shard filter. + * @throws Exception + */ @Test public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTrimHorizon() throws Exception { ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_TRIM_HORIZON).build(); testEmptyLeaseTableBootstrapUsesListShardsWithFilter(INITIAL_POSITION_TRIM_HORIZON, shardFilter); } + /** + * Tests that when reading from AT_TIMESTAMP, we use the AT_TIMESTAMP shard filter. + * @throws Exception + */ @Test public void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTimestamp() throws Exception { ShardFilter shardFilter = ShardFilter.builder().type(ShardFilterType.AT_TIMESTAMP).timestamp(new Date(1000L).toInstant()).build(); @@ -1668,7 +1680,7 @@ public class HierarchicalShardSyncerTest { verify(shardDetector, atLeast(1)).listShards(); } -// /** +// /**getShardFilterFromInitialPosition // * Test CheckIfDescendantAndAddNewLeasesForAncestors - two parents, there is a lease for one parent. // */ // @Test From 05f2002596df644c9761e7e44b5c9d611153ed9e Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Thu, 26 Mar 2020 05:25:20 -0400 Subject: [PATCH 035/159] Fixing unit tests. --- .../amazon/kinesis/leases/HierarchicalShardSyncerTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 d8684cd7..a6de878f 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 @@ -1644,12 +1644,11 @@ public class HierarchicalShardSyncerTest { 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))); - final List currentLeases = Collections.emptyList(); + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, + ShardObjectHelper.newSequenceNumberRange("1", null), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, ShardObjectHelper.MAX_HASH_KEY))); when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); - when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(currentLeases); + when(shardDetector.listShardsWithFilter(shardFilter)).thenReturn(shards); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, From 6a0c17745a7932651970952f9f4e8f52275f7310 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 30 Mar 2020 17:50:33 -0400 Subject: [PATCH 036/159] PR Feedback --- .../leases/HierarchicalShardSyncer.java | 24 ++++----- .../amazon/kinesis/leases/ShardSyncTask.java | 3 +- .../leases/exceptions/ShardSyncer.java | 2 +- .../kinesis/lifecycle/ShutdownTask.java | 3 +- .../leases/HierarchicalShardSyncerTest.java | 54 ++++++++++--------- .../kinesis/lifecycle/ShutdownTaskTest.java | 3 +- 6 files changed, 47 insertions(+), 42 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 d384bfe5..ae017e19 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 @@ -102,23 +102,23 @@ public class HierarchicalShardSyncer { public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope) + final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - final List latestShards = leaseRefresher.isLeaseTableEmpty() ? + final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, garbageCollectLeases, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); + cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, isLeaseTableEmpty, latestShards); } //Provide a pre-collcted list of shards to avoid calling ListShards API public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards) + final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty, + List latestShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 - final boolean isLeaseTableEmpty = leaseRefresher.isLeaseTableEmpty(); if (!CollectionUtils.isNullOrEmpty(latestShards)) { log.debug("Num shards: {}", latestShards.size()); @@ -341,17 +341,15 @@ public class HierarchicalShardSyncer { final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); final Optional> shards = Optional.of(shardDetector.listShardsWithFilter(shardFilter)); - return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream is not in ACTIVE OR UPDATING state - " + - "will retry getting the shard list.")); + return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream " + shardDetector.streamIdentifier().streamName() + + " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); } private static List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { - final List shards = shardDetector.listShards(); - if (shards == null) { - throw new KinesisClientLibIOException( - "Stream is not in ACTIVE OR UPDATING state - will retry getting the shard list."); - } - return shards; + final Optional> shards = Optional.of(shardDetector.listShards()); + + return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream " + shardDetector.streamIdentifier().streamName() + + " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); } /** 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 9291abf4..995d37e5 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 @@ -68,7 +68,8 @@ public class ShardSyncTask implements ConsumerTask { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, - initialPosition, cleanupLeasesUponShardCompletion, garbageCollectLeases, ignoreUnexpectedChildShards, scope); + initialPosition, cleanupLeasesUponShardCompletion, garbageCollectLeases, ignoreUnexpectedChildShards, + scope, leaseRefresher.isLeaseTableEmpty()); if (shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); 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 a14ec784..683b29b3 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 @@ -41,6 +41,6 @@ public class ShardSyncer { final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - true, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope); + true, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, leaseRefresher.isLeaseTableEmpty()); } } 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 55687a1e..8f9003c1 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 @@ -155,7 +155,8 @@ public class ShutdownTask implements ConsumerTask { log.debug("Looking for child shards of shard {}", shardInfo.shardId()); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards); + initialPositionInStream, false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, + scope, false, latestShards); log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); } 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 a6de878f..3a97f1ce 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 @@ -284,7 +284,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -318,7 +318,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -360,7 +360,8 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, + SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty(), latestShards); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -396,7 +397,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, latestShards); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty(), latestShards); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -432,7 +434,8 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, new ArrayList()); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty(), new ArrayList()); final Set expectedShardIds = new HashSet<>(); @@ -472,7 +475,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -492,7 +495,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -527,7 +531,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -572,7 +576,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -628,7 +632,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -643,7 +647,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases present, with shardId-0 being at ShardEnd causing cleanup. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, 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) @@ -703,7 +707,7 @@ public class HierarchicalShardSyncerTest { // Initial call: Call to create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -720,7 +724,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete fails. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { List deleteLeases = leaseDeleteCaptor.getAllValues(); Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -744,7 +748,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete passes. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); deleteLeases = leaseDeleteCaptor.getAllValues(); @@ -805,7 +809,7 @@ public class HierarchicalShardSyncerTest { // Initial call: Call to create leases. Fails on ListLeases hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -815,7 +819,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases not present, leases will be created. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); @@ -830,7 +834,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases present, belongs to TestOwner, shardId-0 is at ShardEnd should be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -896,7 +900,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. Create lease Fails hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -905,7 +909,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); @@ -920,7 +924,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases are present, shardId-0 is at ShardEnd needs to be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -1010,7 +1014,7 @@ public class HierarchicalShardSyncerTest { doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); @@ -1039,7 +1043,7 @@ public class HierarchicalShardSyncerTest { doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); setupMultiStream(); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); @@ -1071,7 +1075,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -1652,7 +1656,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, atLeast(1)).listShardsWithFilter(shardFilter); verify(shardDetector, never()).listShards(); @@ -1674,7 +1678,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, atLeast(1)).listShards(); } 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 927fbf4f..3a34985d 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 @@ -19,6 +19,7 @@ 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.Matchers.anyBoolean; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; @@ -138,7 +139,7 @@ public class ShutdownTaskTest { }).when(hierarchicalShardSyncer) .checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, INITIAL_POSITION_TRIM_HORIZON, false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, - NULL_METRICS_FACTORY.createMetrics(), latestShards); + NULL_METRICS_FACTORY.createMetrics(), false, latestShards); final TaskResult result = task.call(); assertNotNull(result.getException()); From f2911f1f0d44ec0fa90bceb43523d066d0317cc8 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 30 Mar 2020 18:19:32 -0400 Subject: [PATCH 037/159] Moving literals to variable for readability --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 5 ++--- .../amazon/kinesis/leases/ShardSyncTaskManager.java | 7 +++++-- .../amazon/kinesis/leases/exceptions/ShardSyncer.java | 3 ++- .../software/amazon/kinesis/lifecycle/ShutdownTask.java | 6 ++++-- 4 files changed, 13 insertions(+), 8 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 ae017e19..8d280d12 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 @@ -107,15 +107,14 @@ public class HierarchicalShardSyncer { final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, garbageCollectLeases, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, isLeaseTableEmpty, latestShards); + cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards, isLeaseTableEmpty); } //Provide a pre-collcted list of shards to avoid calling ListShards API public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty, - List latestShards) + final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 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 3933b70a..a52ac650 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 @@ -48,6 +48,7 @@ public class ShardSyncTaskManager { @NonNull private final InitialPositionInStreamExtended initialPositionInStream; private final boolean cleanupLeasesUponShardCompletion; + private final boolean garbageCollectLeases; private final boolean ignoreUnexpectedChildShards; private final long shardSyncIdleTimeMillis; @NonNull @@ -84,6 +85,7 @@ public class ShardSyncTaskManager { this.leaseRefresher = leaseRefresher; this.initialPositionInStream = initialPositionInStream; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; + this.garbageCollectLeases = true; this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis; this.executorService = executorService; @@ -114,6 +116,7 @@ public class ShardSyncTaskManager { this.leaseRefresher = leaseRefresher; this.initialPositionInStream = initialPositionInStream; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; + this.garbageCollectLeases = true; this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis; this.executorService = executorService; @@ -128,7 +131,7 @@ public class ShardSyncTaskManager { leaseRefresher, initialPositionInStream, cleanupLeasesUponShardCompletion, - true, + garbageCollectLeases, ignoreUnexpectedChildShards, shardSyncIdleTimeMillis, hierarchicalShardSyncer, @@ -167,7 +170,7 @@ public class ShardSyncTaskManager { leaseRefresher, initialPositionInStream, cleanupLeasesUponShardCompletion, - true, + garbageCollectLeases, ignoreUnexpectedChildShards, shardSyncIdleTimeMillis, hierarchicalShardSyncer, 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 683b29b3..14cd40c3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java @@ -19,6 +19,7 @@ import software.amazon.kinesis.metrics.MetricsScope; @Deprecated public class ShardSyncer { private static final HierarchicalShardSyncer HIERARCHICAL_SHARD_SYNCER = new HierarchicalShardSyncer(); + private static final boolean garbageCollectLeases = true; /** *

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

@@ -41,6 +42,6 @@ public class ShardSyncer { final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - true, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, leaseRefresher.isLeaseTableEmpty()); + garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, leaseRefresher.isLeaseTableEmpty()); } } 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 8f9003c1..7fc4a110 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 @@ -69,6 +69,8 @@ public class ShutdownTask implements ConsumerTask { @NonNull private final InitialPositionInStreamExtended initialPositionInStream; private final boolean cleanupLeasesOfCompletedShards; + private final boolean garbageCollectLeases = false; + private final boolean isLeaseTableEmpty= false; private final boolean ignoreUnexpectedChildShards; @NonNull private final LeaseCoordinator leaseCoordinator; @@ -155,8 +157,8 @@ public class ShutdownTask implements ConsumerTask { log.debug("Looking for child shards of shard {}", shardInfo.shardId()); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, - scope, false, latestShards); + initialPositionInStream, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, + scope, latestShards, isLeaseTableEmpty); log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); } From 82cc3625453ae17b2e884835dc999c2779cd849c Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 30 Mar 2020 18:37:30 -0400 Subject: [PATCH 038/159] --amend --- .../leases/HierarchicalShardSyncer.java | 16 +++--- .../amazon/kinesis/leases/ShardSyncTask.java | 4 +- .../leases/exceptions/ShardSyncer.java | 2 +- .../kinesis/lifecycle/ShutdownTask.java | 8 +-- .../leases/HierarchicalShardSyncerTest.java | 57 +++++++++---------- .../kinesis/lifecycle/ShutdownTaskTest.java | 9 +-- 6 files changed, 46 insertions(+), 50 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 8d280d12..151ead7c 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 @@ -89,10 +89,10 @@ public class HierarchicalShardSyncer { * @param shardDetector * @param leaseRefresher * @param initialPosition - * @param garbageCollectLeases + * @param scope * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards - * @param scope + * @param garbageCollectLeases * @throws DependencyException * @throws InvalidStateException * @throws ProvisionedThroughputException @@ -101,20 +101,20 @@ public class HierarchicalShardSyncer { // CHECKSTYLE:OFF CyclomaticComplexity public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty) + final MetricsScope scope, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, + final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); - checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, garbageCollectLeases, - cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, latestShards, isLeaseTableEmpty); + checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, garbageCollectLeases, + isLeaseTableEmpty); } //Provide a pre-collcted list of shards to avoid calling ListShards API public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final boolean garbageCollectLeases, final boolean cleanupLeasesOfCompletedShards, - final boolean ignoreUnexpectedChildShards, final MetricsScope scope, List latestShards, final boolean isLeaseTableEmpty) + List latestShards, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, + final MetricsScope scope, final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 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 995d37e5..f5c7ab8a 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 @@ -68,8 +68,8 @@ public class ShardSyncTask implements ConsumerTask { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, - initialPosition, cleanupLeasesUponShardCompletion, garbageCollectLeases, ignoreUnexpectedChildShards, - scope, leaseRefresher.isLeaseTableEmpty()); + initialPosition, scope, garbageCollectLeases, ignoreUnexpectedChildShards, cleanupLeasesUponShardCompletion, + leaseRefresher.isLeaseTableEmpty()); if (shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); 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 14cd40c3..c0ed2d2a 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 @@ -42,6 +42,6 @@ public class ShardSyncer { final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, leaseRefresher.isLeaseTableEmpty()); + scope, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, leaseRefresher.isLeaseTableEmpty()); } } 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 7fc4a110..f4538be6 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -16,7 +16,6 @@ package software.amazon.kinesis.lifecycle; import com.google.common.annotations.VisibleForTesting; -import com.sun.org.apache.bcel.internal.generic.LUSHR; import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -28,7 +27,6 @@ import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; -import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; @@ -42,8 +40,6 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.ArrayList; -import java.util.Collection; import java.util.List; /** @@ -157,8 +153,8 @@ public class ShutdownTask implements ConsumerTask { log.debug("Looking for child shards of shard {}", shardInfo.shardId()); // create leases for the child shards hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, - scope, latestShards, isLeaseTableEmpty); + initialPositionInStream, latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, garbageCollectLeases, + isLeaseTableEmpty); log.debug("Finished checking for child shards of shard {}", shardInfo.shardId()); } 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 3a97f1ce..d881b776 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 @@ -30,7 +30,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.math.BigInteger; -import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -284,7 +283,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -318,7 +317,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -360,8 +359,8 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, - SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty(), latestShards); + latestShards, cleanupLeasesOfCompletedShards, false, SCOPE, garbageCollectLeases, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -397,8 +396,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, - dynamoDBLeaseRefresher.isLeaseTableEmpty(), latestShards); + latestShards, cleanupLeasesOfCompletedShards, false, SCOPE, garbageCollectLeases, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -434,8 +433,8 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, - dynamoDBLeaseRefresher.isLeaseTableEmpty(), new ArrayList()); + new ArrayList(), cleanupLeasesOfCompletedShards, false, SCOPE, garbageCollectLeases, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>(); @@ -475,7 +474,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -495,7 +494,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, + INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); @@ -531,7 +530,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, true, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -576,7 +575,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, true, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, true, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -632,7 +631,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -647,7 +646,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases present, with shardId-0 being at ShardEnd causing cleanup. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, 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) @@ -707,7 +706,7 @@ public class HierarchicalShardSyncerTest { // Initial call: Call to create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -724,7 +723,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete fails. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { List deleteLeases = leaseDeleteCaptor.getAllValues(); Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -748,7 +747,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete passes. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); deleteLeases = leaseDeleteCaptor.getAllValues(); @@ -809,7 +808,7 @@ public class HierarchicalShardSyncerTest { // Initial call: Call to create leases. Fails on ListLeases hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -819,7 +818,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases not present, leases will be created. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); @@ -834,7 +833,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases present, belongs to TestOwner, shardId-0 is at ShardEnd should be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -900,7 +899,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. Create lease Fails hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -909,7 +908,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); @@ -924,7 +923,7 @@ public class HierarchicalShardSyncerTest { // Final call: Leases are present, shardId-0 is at ShardEnd needs to be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -1014,7 +1013,7 @@ public class HierarchicalShardSyncerTest { doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); @@ -1043,7 +1042,7 @@ public class HierarchicalShardSyncerTest { doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); setupMultiStream(); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); @@ -1075,7 +1074,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - garbageCollectLeases, cleanupLeasesOfCompletedShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -1656,7 +1655,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, atLeast(1)).listShardsWithFilter(shardFilter); verify(shardDetector, never()).listShards(); @@ -1678,7 +1677,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - garbageCollectLeases, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, atLeast(1)).listShards(); } 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 3a34985d..220fe4a5 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 @@ -19,7 +19,6 @@ 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.Matchers.anyBoolean; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; @@ -44,7 +43,6 @@ import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; -import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; @@ -129,6 +127,9 @@ public class ShutdownTaskTest { */ @Test public final void testCallWhenSyncingShardsThrows() throws Exception { + final boolean garbageCollectLeases = false; + final boolean isLeaseTableEmpty = false; + List latestShards = constructShardListGraphA(); when(shardDetector.listShards()).thenReturn(latestShards); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); @@ -138,8 +139,8 @@ public class ShutdownTaskTest { throw new KinesisClientLibIOException("KinesisClientLibIOException"); }).when(hierarchicalShardSyncer) .checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, INITIAL_POSITION_TRIM_HORIZON, - false, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, - NULL_METRICS_FACTORY.createMetrics(), false, latestShards); + latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, + NULL_METRICS_FACTORY.createMetrics(), garbageCollectLeases, isLeaseTableEmpty); final TaskResult result = task.call(); assertNotNull(result.getException()); From 201a1e1553a25e2d8f4ecbd867c1f016c3d865fb Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 30 Mar 2020 18:46:47 -0400 Subject: [PATCH 039/159] java docs --- .../leases/HierarchicalShardSyncer.java | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 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 151ead7c..71567921 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 @@ -315,7 +315,7 @@ public class HierarchicalShardSyncer { /** * Helper method to resolve the correct shard filter to use when listing shards from a position in a stream. * @param initialPositionInStreamExtended - * @return + * @return ShardFilter shard filter for the corresponding position in the stream. */ private static ShardFilter getShardFilterFromInitialPosition(InitialPositionInStreamExtended initialPositionInStreamExtended) { @@ -361,6 +361,7 @@ public class HierarchicalShardSyncer { * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * location in the shard (when an application starts up for the first time - and there are no checkpoints). * @param inconsistentShardIds Set of child shard ids having open parents. + * @param multiStreamArgs determines if we are using multistream mode. * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, @@ -369,6 +370,18 @@ public class HierarchicalShardSyncer { return leaseSynchronizer.determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds, multiStreamArgs); } + /** + * Determine new leases to create and their initial checkpoint. + * Note: Package level access only for testing purposes. + * + * @param leaseSynchronizer determines the strategy we'll be using to update any new leases. + * @param shards List of all shards in Kinesis (we'll create new leases based on this set) + * @param currentLeases List of current leases + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that + * location in the shard (when an application starts up for the first time - and there are no checkpoints). + * @param inconsistentShardIds Set of child shard ids having open parents. + * @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) { return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds, @@ -378,6 +391,13 @@ public class HierarchicalShardSyncer { /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. + * + * @param leaseSynchronizer determines the strategy we'll be using to update any new leases. + * @param shards List of all shards in Kinesis (we'll create new leases based on this set) + * @param currentLeases List of current leases + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that + * location in the shard (when an application starts up for the first time - and there are no checkpoints). + * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ static List determineNewLeasesToCreate(final LeaseSynchronizer leaseSynchronizer, final List shards, final List currentLeases, final InitialPositionInStreamExtended initialPosition) { @@ -392,7 +412,6 @@ public class HierarchicalShardSyncer { * See javadoc of determineNewLeasesToCreate() for rules and example. * * @param shardId The shardId to check. - * @param shardId The shardId to check. * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * location in the shard (when an application starts up for the first time - and there are no checkpoints). * @param shardIdsOfCurrentLeases The shardIds for the current leases. @@ -812,7 +831,7 @@ public class HierarchicalShardSyncer { * Interface to determine how to create new leases. */ @VisibleForTesting - static interface LeaseSynchronizer { + interface LeaseSynchronizer { /** * Determines how to create leases. * @param shards From 771bc914ebc976a23cb9fc8bb31bda3ade1ffe92 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 31 Mar 2020 15:40:43 -0700 Subject: [PATCH 040/159] Added unit test cases and addressed review comments --- .../InitialPositionInStreamExtended.java | 3 +- .../amazon/kinesis/common/StreamConfig.java | 1 - .../amazon/kinesis/coordinator/Scheduler.java | 15 ++- .../kinesis/coordinator/SchedulerTest.java | 117 ++++++++++++++++++ 4 files changed, 129 insertions(+), 7 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java index 437abf28..b3bedd88 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/InitialPositionInStreamExtended.java @@ -14,6 +14,7 @@ */ package software.amazon.kinesis.common; +import lombok.EqualsAndHashCode; import lombok.ToString; import java.util.Date; @@ -22,7 +23,7 @@ import java.util.Date; * Class that houses the entities needed to specify the position in the stream from where a new application should * start. */ -@ToString +@ToString @EqualsAndHashCode public class InitialPositionInStreamExtended { private final InitialPositionInStream position; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java index 3cf0eeb2..999182b6 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 @@ -6,7 +6,6 @@ import lombok.experimental.Accessors; @Value @Accessors(fluent = true) public class StreamConfig { - // TODO: Consider having streamIdentifier as the unique identifier of this class. StreamIdentifier streamIdentifier; InitialPositionInStreamExtended initialPositionInStreamExtended; } 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 945700a7..3f496f64 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 @@ -98,7 +98,7 @@ import software.amazon.kinesis.retrieval.RetrievalConfig; @Slf4j public class Scheduler implements Runnable { - private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 10000L; + private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; private SchedulerLog slog = new SchedulerLog(); @@ -418,11 +418,12 @@ public class Scheduler implements Runnable { * Sync all streams method. * @return streams that are being synced by this worker */ - private Set checkAndSyncStreamShardsAndLeases() + @VisibleForTesting + Set checkAndSyncStreamShardsAndLeases() throws DependencyException, ProvisionedThroughputException, InvalidStateException { final Set streamsSynced = new HashSet<>(); - if (isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS)) { + if (shouldSyncStreamsNow()) { final Map newStreamConfigMap = new HashMap<>(); // Making an immutable copy newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() @@ -462,7 +463,12 @@ public class Scheduler implements Runnable { return streamsSynced; } - private Set syncStreamsFromLeaseTableOnAppInit() + @VisibleForTesting + boolean shouldSyncStreamsNow() { + return isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); + } + + private void syncStreamsFromLeaseTableOnAppInit() throws DependencyException, ProvisionedThroughputException, InvalidStateException { if (!leasesSyncedOnAppInit && isMultiStreamMode) { final Set streamIdentifiers = leaseCoordinator.leaseRefresher().listLeases().stream() @@ -475,7 +481,6 @@ public class Scheduler implements Runnable { } leasesSyncedOnAppInit = true; } - return Collections.emptySet(); } // When a stream is no longer needed to be tracked, return a default StreamConfig with LATEST for faster shard end. 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 425af67f..ff6eab1c 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 @@ -40,14 +40,20 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import com.google.common.base.Joiner; +import com.google.common.collect.Sets; import io.reactivex.plugins.RxJavaPlugins; import lombok.RequiredArgsConstructor; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -58,6 +64,7 @@ import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.utils.Either; +import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; @@ -76,6 +83,7 @@ import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseRefresher; import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.LifecycleConfig; import software.amazon.kinesis.lifecycle.ShardConsumer; @@ -375,6 +383,115 @@ public class SchedulerTest { } + @Test + public final void testMultiStreamNoStreamsAreSyncedWhenStreamsAreNotRefreshed() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Assert.assertTrue("SyncedStreams should be empty", syncedStreams.isEmpty()); + Assert.assertEquals(new HashSet(streamConfigList1), new HashSet(scheduler.currentStreamConfigMap().values())); + } + + @Test + public final void testMultiStreamOnlyNewStreamsAreSynced() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.range(5, 7).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(streamConfigList2), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + } + + @Test + public final void testMultiStreamOnlyStaleStreamsAreSynced() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(streamConfigList2), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + } + + @Test + public final void testMultiStreamSyncOnlyNewAndStaleStreamsAreSynced() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(streamConfigList2), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + } + @Test public final void testSchedulerShutdown() { scheduler.shutdown(); From 2f9ce0ce4f7c83025f8e33376ef1b326789f1a83 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 5 Mar 2020 16:44:37 -0800 Subject: [PATCH 041/159] ShardEnd Shard Sync --- .../leases/HierarchicalShardSyncer.java | 13 ++ .../kinesis/lifecycle/ConsumerStates.java | 3 +- .../kinesis/lifecycle/ShardConsumer.java | 7 +- .../kinesis/lifecycle/ShutdownTask.java | 91 +++++++------ .../lifecycle/events/ProcessRecordsInput.java | 6 + .../fanout/FanOutRecordsPublisher.java | 38 ++++-- .../polling/BlockingRecordsPublisher.java | 1 + .../retrieval/polling/KinesisDataFetcher.java | 22 +++- .../polling/PrefetchRecordsPublisher.java | 3 +- .../kinesis/lifecycle/ConsumerStatesTest.java | 29 ++++- .../kinesis/lifecycle/ShutdownTaskTest.java | 122 +++++++----------- .../fanout/FanOutRecordsPublisherTest.java | 119 ++++++++++++++++- .../polling/KinesisDataFetcherTest.java | 29 ++++- ...efetchRecordsPublisherIntegrationTest.java | 5 +- .../polling/PrefetchRecordsPublisherTest.java | 68 +++++++++- .../utils/ProcessRecordsInputMatcher.java | 1 + 16 files changed, 405 insertions(+), 152 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 ecd64952..34c17bdf 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 @@ -39,6 +39,7 @@ import org.apache.commons.lang3.StringUtils; import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.services.kinesis.model.ShardFilterType; @@ -164,6 +165,18 @@ public class HierarchicalShardSyncer { } } + public synchronized Lease createLeaseForChildShard(ChildShard childShard) throws InvalidStateException { + Lease newLease = new Lease(); + newLease.leaseKey(childShard.shardId()); + if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + newLease.parentShardIds(childShard.parentShards()); + } else { + throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found."); + } + newLease.ownerSwitchesSinceCheckpoint(0L); + return newLease; + } + // CHECKSTYLE:ON CyclomaticComplexity /** Note: This method has package level access solely for testing purposes. diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java index bb1788b2..58e31985 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 @@ -496,7 +496,8 @@ class ConsumerStates { argument.taskBackoffTimeMillis(), argument.recordsPublisher(), argument.hierarchicalShardSyncer(), - argument.metricsFactory()); + argument.metricsFactory(), + input == null ? null : input.childShards()); } @Override 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 e34f2ea4..b6e7c068 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumer.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.lifecycle; import java.time.Duration; import java.time.Instant; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -32,6 +33,7 @@ import lombok.Getter; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; import software.amazon.kinesis.leases.ShardInfo; @@ -86,6 +88,8 @@ public class ShardConsumer { private final ShardConsumerSubscriber subscriber; + private ProcessRecordsInput shardEndProcessRecordsInput; + @Deprecated public ShardConsumer(RecordsPublisher recordsPublisher, ExecutorService executorService, ShardInfo shardInfo, Optional logWarningForTaskAfterMillis, ShardConsumerArgument shardConsumerArgument, @@ -148,6 +152,7 @@ public class ShardConsumer { processData(input); if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.SHARD_END); + shardEndProcessRecordsInput = input; subscription.cancel(); return; } @@ -305,7 +310,7 @@ public class ShardConsumer { return true; } - executeTask(null); + executeTask(shardEndProcessRecordsInput); return false; } }, executorService); 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 18e2be76..18a0af63 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 @@ -20,6 +20,7 @@ import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -30,7 +31,11 @@ import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; +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.LeaseLostInput; +import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsScope; @@ -81,6 +86,8 @@ public class ShutdownTask implements ConsumerTask { private final TaskType taskType = TaskType.SHUTDOWN; + private final List childShards; + private static final Function shardInfoIdProvider = shardInfo -> shardInfo .streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()).orElse(shardInfo.shardId()); /* @@ -99,67 +106,48 @@ public class ShutdownTask implements ConsumerTask { try { try { - ShutdownReason localReason = reason; - List latestShards = null; - /* - * Revalidate if the current shard is closed before shutting down the shard consumer with reason SHARD_END - * If current shard is not closed, shut down the shard consumer with reason LEASE_LOST that allows - * active workers to contend for the lease of this shard. - */ - if (localReason == ShutdownReason.SHARD_END) { - latestShards = shardDetector.listShards(); + log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", + shardInfoIdProvider.apply(shardInfo), shardInfo.concurrencyToken(), reason); - //If latestShards is empty, should also shutdown the ShardConsumer without checkpoint with SHARD_END - if (CollectionUtils.isNullOrEmpty(latestShards) || !isShardInContextParentOfAny(latestShards)) { - localReason = ShutdownReason.LEASE_LOST; - dropLease(); - log.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfoIdProvider.apply(shardInfo)); + final long startTime = System.currentTimeMillis(); + if (reason == ShutdownReason.SHARD_END) { + // Create new lease for the child shards if they don't exist. + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); } - } - // If we reached end of the shard, set sequence number to SHARD_END. - if (localReason == ShutdownReason.SHARD_END) { recordProcessorCheckpointer .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); - } - - log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", - shardInfoIdProvider.apply(shardInfo), shardInfo.concurrencyToken(), localReason); - final ShutdownInput shutdownInput = ShutdownInput.builder().shutdownReason(localReason) - .checkpointer(recordProcessorCheckpointer).build(); - final long startTime = System.currentTimeMillis(); - try { - if (localReason == ShutdownReason.SHARD_END) { + // Call the shardReocrdsProcessor 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. + try { shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); + final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); if (lastCheckpointValue == null || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfoIdProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + - "See ShardRecordProcessor.shardEnded javadocs for more information."); + + shardInfoIdProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + + "See ShardRecordProcessor.shardEnded javadocs for more information."); } - } else { - shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()); + } catch (Exception e) { + applicationException = true; + throw e; + } finally { + MetricsUtil.addLatency(scope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); + } + } else { + try { + shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()); + } catch (Exception e) { + applicationException = true; + throw e; } - log.debug("Shutting down retrieval strategy."); - recordsPublisher.shutdown(); - log.debug("Record processor completed shutdown() for shard {}", shardInfoIdProvider.apply(shardInfo)); - } catch (Exception e) { - applicationException = true; - throw e; - } finally { - MetricsUtil.addLatency(scope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); } - if (localReason == ShutdownReason.SHARD_END) { - log.debug("Looking for child shards of shard {}", shardInfoIdProvider.apply(shardInfo)); - // create leases for the child shards - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseCoordinator.leaseRefresher(), - initialPositionInStream, latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, garbageCollectLeases, - isLeaseTableEmpty); - log.debug("Finished checking for child shards of shard {}", shardInfoIdProvider.apply(shardInfo)); - } + log.debug("Shutting down retrieval strategy."); + recordsPublisher.shutdown(); + log.debug("Record processor completed shutdown() for shard {}", shardInfoIdProvider.apply(shardInfo)); return new TaskResult(null); } catch (Exception e) { @@ -181,7 +169,16 @@ public class ShutdownTask implements ConsumerTask { } return new TaskResult(exception); + } + private void createLeasesForChildShardsIfNotExist() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + for(ChildShard childShard : childShards) { + if(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId()) == null) { + final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard); + leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); + } + } } /* diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java index b7dd4e05..3bfcd514 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java @@ -23,6 +23,7 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.ToString; import lombok.experimental.Accessors; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.processor.RecordProcessorCheckpointer; import software.amazon.kinesis.retrieval.KinesisClientRecord; @@ -66,6 +67,11 @@ public class ProcessRecordsInput { * This value does not include the {@link #timeSpentInCache()}. */ private Long millisBehindLatest; + /** + * A list of child shards if the current GetRecords request reached the shard end. + * If not at the shard end, this should be an empty list. + */ + private List childShards; /** * How long the records spent waiting to be dispatched to the {@link ShardRecordProcessor} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index c24a3803..38075890 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 @@ -33,11 +33,13 @@ import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponse; import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.awssdk.utils.Either; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.KinesisRequestsBuilder; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.retrieval.BatchUniqueIdentifier; import software.amazon.kinesis.retrieval.IteratorBuilder; @@ -398,7 +400,7 @@ 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).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(); @@ -477,15 +479,28 @@ public class FanOutRecordsPublisher implements RecordsPublisher { return; } - List records = recordBatchEvent.records().stream().map(KinesisClientRecord::fromRecord) - .collect(Collectors.toList()); - ProcessRecordsInput input = ProcessRecordsInput.builder().cacheEntryTime(Instant.now()) - .millisBehindLatest(recordBatchEvent.millisBehindLatest()) - .isAtShardEnd(recordBatchEvent.continuationSequenceNumber() == null).records(records).build(); - FanoutRecordsRetrieved recordsRetrieved = new FanoutRecordsRetrieved(input, - recordBatchEvent.continuationSequenceNumber(), triggeringFlow.subscribeToShardId); - try { + // If recordBatchEvent is not valid event, RuntimeException will be thrown here and trigger the errorOccurred call. + // Since the triggeringFlow is active flow, it will then trigger the handleFlowError call. + // Since the exception is not ResourceNotFoundException, it will trigger onError in the ShardConsumerSubscriber. + // The ShardConsumerSubscriber will finally cancel the subscription. + if (!isValidEvent(recordBatchEvent)) { + throw new InvalidStateException("RecordBatchEvent for flow " + triggeringFlow.toString() + " is invalid." + + " event.continuationSequenceNumber: " + recordBatchEvent.continuationSequenceNumber() + + ". event.childShards: " + recordBatchEvent.childShards()); + } + + List records = recordBatchEvent.records().stream().map(KinesisClientRecord::fromRecord) + .collect(Collectors.toList()); + ProcessRecordsInput input = ProcessRecordsInput.builder() + .cacheEntryTime(Instant.now()) + .millisBehindLatest(recordBatchEvent.millisBehindLatest()) + .isAtShardEnd(recordBatchEvent.continuationSequenceNumber() == null) + .records(records) + .childShards(recordBatchEvent.childShards()) + .build(); + FanoutRecordsRetrieved recordsRetrieved = new FanoutRecordsRetrieved(input, + recordBatchEvent.continuationSequenceNumber(), triggeringFlow.subscribeToShardId); bufferCurrentEventAndScheduleIfRequired(recordsRetrieved, triggeringFlow); } catch (Throwable t) { log.warn("{}: Unable to buffer or schedule onNext for subscriber. Failing publisher." + @@ -495,6 +510,11 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } } + private boolean isValidEvent(SubscribeToShardEvent event) { + return event.continuationSequenceNumber() == null ? !CollectionUtils.isNullOrEmpty(event.childShards()) + : event.childShards() != null && event.childShards().isEmpty(); + } + private void updateAvailableQueueSpaceAndRequestUpstream(RecordFlow triggeringFlow) { if (availableQueueSpace <= 0) { log.debug( diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/BlockingRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/BlockingRecordsPublisher.java index 1e6462f5..33be11d4 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/BlockingRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/BlockingRecordsPublisher.java @@ -67,6 +67,7 @@ public class BlockingRecordsPublisher implements RecordsPublisher { return ProcessRecordsInput.builder() .records(records) .millisBehindLatest(getRecordsResult.millisBehindLatest()) + .childShards(getRecordsResult.childShards()) .build(); } 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 1605f941..dc25b20c 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 @@ -36,6 +36,7 @@ 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; @@ -120,7 +121,13 @@ public class KinesisDataFetcher { if (nextIterator != null) { try { - return new AdvancingResult(getRecords(nextIterator)); + GetRecordsResponse getRecordsResponse = getRecords(nextIterator); + while (!isValidResponse(getRecordsResponse)) { + log.error("{} : GetRecords response is not valid. nextShardIterator: {}. childShards: {}. Will retry GetRecords with the same nextIterator.", + shardId, getRecordsResponse.nextShardIterator(), getRecordsResponse.childShards()); + getRecordsResponse = getRecords(nextIterator); + } + return new AdvancingResult(getRecordsResponse); } catch (ResourceNotFoundException e) { log.info("Caught ResourceNotFoundException when fetching records for shard {}", streamAndShardId); return TERMINAL_RESULT; @@ -133,8 +140,12 @@ public class KinesisDataFetcher { final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() { @Override public GetRecordsResponse getResult() { - return GetRecordsResponse.builder().millisBehindLatest(null).records(Collections.emptyList()) - .nextShardIterator(null).build(); + return GetRecordsResponse.builder() + .millisBehindLatest(null) + .records(Collections.emptyList()) + .nextShardIterator(null) + .childShards(Collections.emptyList()) + .build(); } @Override @@ -177,6 +188,11 @@ public class KinesisDataFetcher { } } + private boolean isValidResponse(GetRecordsResponse response) { + return response.nextShardIterator() == null ? !CollectionUtils.isNullOrEmpty(response.childShards()) + : response.childShards() != null && response.childShards().isEmpty(); + } + /** * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number. * @param initialCheckpoint Current checkpoint sequence number for this shard. 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 ba8aa117..6e172f08 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 @@ -162,7 +162,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { } else { log.info( "{}: No record batch found while evicting from the prefetch queue. This indicates the prefetch buffer" - + "was reset.", streamAndShardId); + + " was reset.", streamAndShardId); } return result; } @@ -437,6 +437,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { .millisBehindLatest(getRecordsResult.millisBehindLatest()) .cacheEntryTime(lastSuccessfulCall) .isAtShardEnd(getRecordsRetrievalStrategy.getDataFetcher().isShardEndReached()) + .childShards(getRecordsResult.childShards()) .build(); PrefetchRecordsRetrieved recordsRetrieved = new PrefetchRecordsRetrieved(processRecordsInput, 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 5d8e302f..b4164f90 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java @@ -25,6 +25,8 @@ import static org.mockito.Mockito.when; import static software.amazon.kinesis.lifecycle.ConsumerStates.ShardConsumerState; import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutorService; @@ -40,6 +42,7 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -49,6 +52,7 @@ import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -57,6 +61,9 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; +import javax.swing.*; +import javax.swing.text.AsyncBoxView; + @RunWith(MockitoJUnitRunner.class) public class ConsumerStatesTest { private static final String STREAM_NAME = "TestStream"; @@ -300,13 +307,27 @@ public class ConsumerStatesTest { } - // TODO: Fix this test - @Ignore @Test public void shuttingDownStateTest() { consumer.markForShutdown(ShutdownReason.SHARD_END); ConsumerState state = ShardConsumerState.SHUTTING_DOWN.consumerState(); - ConsumerTask task = state.createTask(argument, consumer, null); + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add("shardId-000000000000"); + ChildShard leftChild = ChildShard.builder() + .shardId("shardId-000000000001") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("shardId-000000000002") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + when(processRecordsInput.childShards()).thenReturn(childShards); + ConsumerTask task = state.createTask(argument, consumer, processRecordsInput); assertThat(task, shutdownTask(ShardInfo.class, "shardInfo", equalTo(shardInfo))); assertThat(task, @@ -315,8 +336,6 @@ public class ConsumerStatesTest { equalTo(recordProcessorCheckpointer))); assertThat(task, shutdownTask(ShutdownReason.class, "reason", equalTo(reason))); assertThat(task, shutdownTask(LeaseCoordinator.class, "leaseCoordinator", equalTo(leaseCoordinator))); - assertThat(task, shutdownTask(InitialPositionInStreamExtended.class, "initialPositionInStream", - equalTo(initialPositionInStream))); assertThat(task, shutdownTask(Boolean.class, "cleanupLeasesOfCompletedShards", equalTo(cleanupLeasesOfCompletedShards))); assertThat(task, shutdownTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java index 220fe4a5..8a9024f6 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 @@ -26,6 +26,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -33,9 +34,11 @@ import java.util.List; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.Matchers; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; @@ -43,11 +46,15 @@ import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; +import software.amazon.kinesis.leases.Lease; 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.ShardObjectHelper; +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.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; import software.amazon.kinesis.metrics.MetricsFactory; @@ -104,7 +111,7 @@ public class ShutdownTaskTest { task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards()); } /** @@ -113,8 +120,8 @@ public class ShutdownTaskTest { */ @Test public final void testCallWhenApplicationDoesNotCheckpoint() { - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); assertNotNull(result.getException()); @@ -126,28 +133,18 @@ public class ShutdownTaskTest { * This test is for the scenario that checkAndCreateLeaseForNewShards throws an exception. */ @Test - public final void testCallWhenSyncingShardsThrows() throws Exception { - final boolean garbageCollectLeases = false; - final boolean isLeaseTableEmpty = false; - - List latestShards = constructShardListGraphA(); - when(shardDetector.listShards()).thenReturn(latestShards); + public final void testCallWhenCreatingNewLeasesThrows() throws Exception { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); - - doAnswer((invocation) -> { - throw new KinesisClientLibIOException("KinesisClientLibIOException"); - }).when(hierarchicalShardSyncer) - .checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, INITIAL_POSITION_TRIM_HORIZON, - latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, - NULL_METRICS_FACTORY.createMetrics(), garbageCollectLeases, isLeaseTableEmpty); + when(leaseRefresher.createLeaseIfNotExists(Matchers.any(Lease.class))).thenThrow(new KinesisClientLibIOException("KinesisClientLibIOException")); final TaskResult result = task.call(); assertNotNull(result.getException()); assertTrue(result.getException() instanceof KinesisClientLibIOException); - verify(recordsPublisher).shutdown(); - verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(recordsPublisher, never()).shutdown(); + verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } /** @@ -155,24 +152,24 @@ public class ShutdownTaskTest { * This test is for the scenario that ShutdownTask is created for ShardConsumer reaching the Shard End. */ @Test - public final void testCallWhenTrueShardEnd() { + public final void testCallWhenTrueShardEnd() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards()); - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); - verify(shardDetector, times(1)).listShards(); - verify(leaseCoordinator, never()).getAssignments(); + verify(leaseRefresher, times(2)).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } /** @@ -180,23 +177,25 @@ public class ShutdownTaskTest { * This test is for the scenario that a ShutdownTask is created for detecting a false Shard End. */ @Test - public final void testCallWhenFalseShardEnd() { + public final void testCallWhenShardNotFound() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>()); - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); + when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); - verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); - verify(shardDetector, times(1)).listShards(); - verify(leaseCoordinator).getCurrentlyHeldLease(shardInfo.shardId()); + verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator, never()).getCurrentlyHeldLease(shardInfo.shardId()); + verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } /** @@ -204,23 +203,22 @@ public class ShutdownTaskTest { * This test is for the scenario that a ShutdownTask is created for the ShardConsumer losing the lease. */ @Test - public final void testCallWhenLeaseLost() { + public final void testCallWhenLeaseLost() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, LEASE_LOST_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY); - - when(shardDetector.listShards()).thenReturn(constructShardListGraphA()); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>()); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); - verify(shardDetector, never()).listShards(); verify(leaseCoordinator, never()).getAssignments(); + verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } /** @@ -231,45 +229,23 @@ public class ShutdownTaskTest { assertEquals(TaskType.SHUTDOWN, task.taskType()); } - - /* - * Helper method to construct a shard list for graph A. Graph A is defined below. Shard structure (y-axis is - * epochs): 0 1 2 3 4 5 - shards till - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - */ - private List constructShardListGraphA() { - final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); - final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); - final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("11", "205"); - final SequenceNumberRange range3 = ShardObjectHelper.newSequenceNumberRange("103", "205"); - final SequenceNumberRange range4 = ShardObjectHelper.newSequenceNumberRange("206", null); - - return Arrays.asList( - ShardObjectHelper.newShard("shardId-0", null, null, range0, - ShardObjectHelper.newHashKeyRange("0", "99")), - ShardObjectHelper.newShard("shardId-1", null, null, range0, - ShardObjectHelper.newHashKeyRange("100", "199")), - ShardObjectHelper.newShard("shardId-2", null, null, range0, - ShardObjectHelper.newHashKeyRange("200", "299")), - ShardObjectHelper.newShard("shardId-3", null, null, range0, - ShardObjectHelper.newHashKeyRange("300", "399")), - ShardObjectHelper.newShard("shardId-4", null, null, range1, - ShardObjectHelper.newHashKeyRange("400", "499")), - ShardObjectHelper.newShard("shardId-5", null, null, range2, - ShardObjectHelper.newHashKeyRange("500", ShardObjectHelper.MAX_HASH_KEY)), - ShardObjectHelper.newShard("shardId-6", "shardId-0", "shardId-1", range3, - ShardObjectHelper.newHashKeyRange("0", "199")), - ShardObjectHelper.newShard("shardId-7", "shardId-2", "shardId-3", range3, - ShardObjectHelper.newHashKeyRange("200", "399")), - ShardObjectHelper.newShard("shardId-8", "shardId-6", "shardId-7", range4, - ShardObjectHelper.newHashKeyRange("0", "399")), - ShardObjectHelper.newShard("shardId-9", "shardId-5", null, range4, - ShardObjectHelper.newHashKeyRange("500", "799")), - ShardObjectHelper.newShard("shardId-10", null, "shardId-5", range4, - ShardObjectHelper.newHashKeyRange("800", ShardObjectHelper.MAX_HASH_KEY))); + private List constructChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(shardId); + ChildShard leftChild = ChildShard.builder() + .shardId("ShardId-1") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("ShardId-2") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + return childShards; } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java index fe6489b9..43881122 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisherTest.java @@ -26,6 +26,7 @@ import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.core.async.SdkPublisher; import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; @@ -35,6 +36,7 @@ import software.amazon.awssdk.services.kinesis.model.SubscribeToShardEventStream import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.lifecycle.ShardConsumerNotifyingSubscriber; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.retrieval.BatchUniqueIdentifier; @@ -47,6 +49,7 @@ import software.amazon.kinesis.utils.SubscribeToShardRequestMatcher; import java.nio.ByteBuffer; import java.time.Instant; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedList; import java.util.List; import java.util.Optional; @@ -89,6 +92,7 @@ public class FanOutRecordsPublisherTest { private static final String SHARD_ID = "Shard-001"; private static final String CONSUMER_ARN = "arn:consumer"; + private static final String CONTINUATION_SEQUENCE_NUMBER = "continuationSequenceNumber"; @Mock private KinesisAsyncClient kinesisClient; @@ -148,7 +152,12 @@ public class FanOutRecordsPublisherTest { List matchers = records.stream().map(KinesisClientRecordMatcher::new) .collect(Collectors.toList()); - batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).build(); + batchEvent = SubscribeToShardEvent.builder() + .millisBehindLatest(100L) + .records(records) + .continuationSequenceNumber("test") + .childShards(Collections.emptyList()) + .build(); captor.getValue().onNext(batchEvent); captor.getValue().onNext(batchEvent); @@ -166,6 +175,73 @@ public class FanOutRecordsPublisherTest { } + @Test + public void InvalidEventTest() throws Exception { + FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN); + + ArgumentCaptor captor = ArgumentCaptor + .forClass(FanOutRecordsPublisher.RecordSubscription.class); + ArgumentCaptor flowCaptor = ArgumentCaptor + .forClass(FanOutRecordsPublisher.RecordFlow.class); + + doNothing().when(publisher).subscribe(captor.capture()); + + source.start(ExtendedSequenceNumber.LATEST, + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST)); + + List receivedInput = new ArrayList<>(); + + source.subscribe(new ShardConsumerNotifyingSubscriber(new Subscriber() { + Subscription subscription; + + @Override public void onSubscribe(Subscription s) { + subscription = s; + subscription.request(1); + } + + @Override public void onNext(RecordsRetrieved input) { + receivedInput.add(input.processRecordsInput()); + subscription.request(1); + } + + @Override public void onError(Throwable t) { + log.error("Caught throwable in subscriber", t); + fail("Caught throwable in subscriber"); + } + + @Override public void onComplete() { + fail("OnComplete called when not expected"); + } + }, source)); + + verify(kinesisClient).subscribeToShard(any(SubscribeToShardRequest.class), flowCaptor.capture()); + flowCaptor.getValue().onEventStream(publisher); + captor.getValue().onSubscribe(subscription); + + List records = Stream.of(1, 2, 3).map(this::makeRecord).collect(Collectors.toList()); + List matchers = records.stream().map(KinesisClientRecordMatcher::new) + .collect(Collectors.toList()); + + batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).continuationSequenceNumber(CONTINUATION_SEQUENCE_NUMBER).build(); + SubscribeToShardEvent invalidEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).childShards(Collections.emptyList()).build(); + + captor.getValue().onNext(batchEvent); + captor.getValue().onNext(invalidEvent); + captor.getValue().onNext(batchEvent); + + // When the second request failed with invalid event, it should stop sending requests and cancel the flow. + verify(subscription, times(2)).request(1); + assertThat(receivedInput.size(), equalTo(1)); + + receivedInput.stream().map(ProcessRecordsInput::records).forEach(clientRecordsList -> { + assertThat(clientRecordsList.size(), equalTo(matchers.size())); + for (int i = 0; i < clientRecordsList.size(); ++i) { + assertThat(clientRecordsList.get(i), matchers.get(i)); + } + }); + + } + @Test public void testIfAllEventsReceivedWhenNoTasksRejectedByExecutor() throws Exception { FanOutRecordsPublisher source = new FanOutRecordsPublisher(kinesisClient, SHARD_ID, CONSUMER_ARN); @@ -225,7 +301,9 @@ public class FanOutRecordsPublisherTest { SubscribeToShardEvent.builder() .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum) - .records(records).build()) + .records(records) + .childShards(Collections.emptyList()) + .build()) .forEach(batchEvent -> captor.getValue().onNext(batchEvent)); verify(subscription, times(4)).request(1); @@ -301,7 +379,9 @@ public class FanOutRecordsPublisherTest { SubscribeToShardEvent.builder() .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum) - .records(records).build()) + .records(records) + .childShards(Collections.emptyList()) + .build()) .forEach(batchEvent -> captor.getValue().onNext(batchEvent)); verify(subscription, times(2)).request(1); @@ -334,6 +414,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -436,6 +517,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -536,13 +618,30 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(SHARD_ID); + ChildShard leftChild = ChildShard.builder() + .shardId("Shard-002") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("Shard-003") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); Consumer servicePublisherShardEndAction = contSeqNum -> captor.getValue().onNext( SubscribeToShardEvent.builder() .millisBehindLatest(100L) .continuationSequenceNumber(null) .records(records) + .childShards(childShards) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -648,6 +747,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -750,6 +850,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(2); @@ -842,6 +943,7 @@ public class FanOutRecordsPublisherTest { .millisBehindLatest(100L) .continuationSequenceNumber(contSeqNum + "") .records(records) + .childShards(Collections.emptyList()) .build()); CountDownLatch servicePublisherTaskCompletionLatch = new CountDownLatch(1); @@ -1004,7 +1106,12 @@ public class FanOutRecordsPublisherTest { List matchers = records.stream().map(KinesisClientRecordMatcher::new) .collect(Collectors.toList()); - batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records).build(); + batchEvent = SubscribeToShardEvent.builder() + .millisBehindLatest(100L) + .records(records) + .continuationSequenceNumber(CONTINUATION_SEQUENCE_NUMBER) + .childShards(Collections.emptyList()) + .build(); captor.getValue().onNext(batchEvent); captor.getValue().onNext(batchEvent); @@ -1098,7 +1205,7 @@ public class FanOutRecordsPublisherTest { .collect(Collectors.toList()); batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(records) - .continuationSequenceNumber("3").build(); + .continuationSequenceNumber("3").childShards(Collections.emptyList()).build(); captor.getValue().onNext(batchEvent); captor.getValue().onComplete(); @@ -1126,7 +1233,7 @@ public class FanOutRecordsPublisherTest { .collect(Collectors.toList()); batchEvent = SubscribeToShardEvent.builder().millisBehindLatest(100L).records(nextRecords) - .continuationSequenceNumber("6").build(); + .continuationSequenceNumber("6").childShards(Collections.emptyList()).build(); nextSubscribeCaptor.getValue().onNext(batchEvent); verify(subscription, times(4)).request(1); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java index a88f3c3b..74b0c125 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/KinesisDataFetcherTest.java @@ -30,6 +30,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Date; @@ -53,6 +54,7 @@ import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; @@ -65,6 +67,7 @@ import software.amazon.kinesis.checkpoint.SentinelCheckpoint; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.exceptions.KinesisClientLibException; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.NullMetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -330,8 +333,31 @@ public class KinesisDataFetcherTest { private CompletableFuture makeGetRecordsResponse(String nextIterator, List records) throws InterruptedException, ExecutionException { + List childShards = new ArrayList<>(); + if(nextIterator == null) { + childShards = createChildShards(); + } return CompletableFuture.completedFuture(GetRecordsResponse.builder().nextShardIterator(nextIterator) - .records(CollectionUtils.isNullOrEmpty(records) ? Collections.emptyList() : records).build()); + .records(CollectionUtils.isNullOrEmpty(records) ? Collections.emptyList() : records).childShards(childShards).build()); + } + + private List createChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(SHARD_ID); + ChildShard leftChild = ChildShard.builder() + .shardId("Shard-2") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("Shard-3") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + return childShards; } @Test @@ -342,6 +368,7 @@ public class KinesisDataFetcherTest { final String initialIterator = "InitialIterator"; final String nextIterator1 = "NextIteratorOne"; final String nextIterator2 = "NextIteratorTwo"; + final String nextIterator3 = "NextIteratorThree"; final CompletableFuture nonAdvancingResult1 = makeGetRecordsResponse(initialIterator, null); final CompletableFuture nonAdvancingResult2 = makeGetRecordsResponse(nextIterator1, null); final CompletableFuture finalNonAdvancingResult = makeGetRecordsResponse(nextIterator2, diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java index f940faf2..461fce71 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherIntegrationTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -48,6 +49,7 @@ import org.mockito.runners.MockitoJUnitRunner; import org.mockito.stubbing.Answer; import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import lombok.extern.slf4j.Slf4j; @@ -86,6 +88,7 @@ public class PrefetchRecordsPublisherIntegrationTest { private String operation = "ProcessTask"; private String streamName = "streamName"; private String shardId = "shardId-000000000000"; + private String nextShardIterator = "testNextShardIterator"; @Mock private KinesisAsyncClient kinesisClient; @@ -249,7 +252,7 @@ public class PrefetchRecordsPublisherIntegrationTest { @Override public DataFetcherResult getRecords() { - GetRecordsResponse getRecordsResult = GetRecordsResponse.builder().records(new ArrayList<>(records)).millisBehindLatest(1000L).build(); + GetRecordsResponse getRecordsResult = GetRecordsResponse.builder().records(new ArrayList<>(records)).nextShardIterator(nextShardIterator).millisBehindLatest(1000L).build(); return new AdvancingResult(getRecordsResult); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java index a28ded63..f7051ec4 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -71,11 +72,13 @@ import io.reactivex.Flowable; import io.reactivex.schedulers.Schedulers; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.awssdk.services.kinesis.model.Record; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.lifecycle.ShardConsumerNotifyingSubscriber; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; @@ -99,6 +102,7 @@ public class PrefetchRecordsPublisherTest { private static final int MAX_SIZE = 5; private static final int MAX_RECORDS_COUNT = 15000; private static final long IDLE_MILLIS_BETWEEN_CALLS = 0L; + private static final String NEXT_SHARD_ITERATOR = "testNextShardIterator"; @Mock private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @@ -136,7 +140,7 @@ public class PrefetchRecordsPublisherTest { "shardId"); spyQueue = spy(getRecordsCache.getPublisherSession().prefetchRecordsQueue()); records = spy(new ArrayList<>()); - getRecordsResponse = GetRecordsResponse.builder().records(records).build(); + getRecordsResponse = GetRecordsResponse.builder().records(records).nextShardIterator(NEXT_SHARD_ITERATOR).childShards(new ArrayList<>()).build(); when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResponse); } @@ -155,11 +159,67 @@ public class PrefetchRecordsPublisherTest { .processRecordsInput(); assertEquals(expectedRecords, result.records()); + assertEquals(new ArrayList<>(), result.childShards()); verify(executorService).execute(any()); verify(getRecordsRetrievalStrategy, atLeast(1)).getRecords(eq(MAX_RECORDS_PER_CALL)); } + @Test + public void testGetRecordsWithInvalidResponse() { + record = Record.builder().data(createByteBufferWithSize(SIZE_512_KB)).build(); + + when(records.size()).thenReturn(1000); + + GetRecordsResponse response = GetRecordsResponse.builder().records(records).build(); + when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(response); + when(dataFetcher.isShardEndReached()).thenReturn(false); + + getRecordsCache.start(sequenceNumber, initialPosition); + + try { + ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L) + .processRecordsInput(); + } catch (Exception e) { + assertEquals("No records found", e.getMessage()); + } + } + + @Test + public void testGetRecordsWithShardEnd() { + records = new ArrayList<>(); + + final List expectedRecords = new ArrayList<>(); + + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add("shardId"); + ChildShard leftChild = ChildShard.builder() + .shardId("shardId-000000000001") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("shardId-000000000002") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + childShards.add(leftChild); + childShards.add(rightChild); + + GetRecordsResponse response = GetRecordsResponse.builder().records(records).childShards(childShards).build(); + when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(response); + when(dataFetcher.isShardEndReached()).thenReturn(true); + + getRecordsCache.start(sequenceNumber, initialPosition); + ProcessRecordsInput result = blockUntilRecordsAvailable(() -> evictPublishedEvent(getRecordsCache, "shardId"), 1000L) + .processRecordsInput(); + + assertEquals(expectedRecords, result.records()); + assertEquals(childShards, result.childShards()); + assertTrue(result.isAtShardEnd()); + } + // TODO: Broken test @Test @Ignore @@ -270,7 +330,7 @@ public class PrefetchRecordsPublisherTest { @Test public void testRetryableRetrievalExceptionContinues() { - GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L).records(Collections.emptyList()).build(); + GetRecordsResponse response = GetRecordsResponse.builder().millisBehindLatest(100L).records(Collections.emptyList()).nextShardIterator(NEXT_SHARD_ITERATOR).build(); when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenThrow(new RetryableRetrievalException("Timeout", new TimeoutException("Timeout"))).thenReturn(response); getRecordsCache.start(sequenceNumber, initialPosition); @@ -293,7 +353,7 @@ public class PrefetchRecordsPublisherTest { when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenAnswer( i -> GetRecordsResponse.builder().records( Record.builder().data(SdkBytes.fromByteArray(new byte[] { 1, 2, 3 })).sequenceNumber(++sequenceNumberInResponse[0] + "").build()) - .build()); + .nextShardIterator(NEXT_SHARD_ITERATOR).build()); getRecordsCache.start(sequenceNumber, initialPosition); @@ -384,7 +444,7 @@ public class PrefetchRecordsPublisherTest { // to the subscriber. GetRecordsResponse response = GetRecordsResponse.builder().records( Record.builder().data(SdkBytes.fromByteArray(new byte[] { 1, 2, 3 })).sequenceNumber("123").build()) - .build(); + .nextShardIterator(NEXT_SHARD_ITERATOR).build(); when(getRecordsRetrievalStrategy.getRecords(anyInt())).thenReturn(response); getRecordsCache.start(sequenceNumber, initialPosition); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java index a89ebef6..1aeddc60 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/ProcessRecordsInputMatcher.java @@ -45,6 +45,7 @@ public class ProcessRecordsInputMatcher extends TypeSafeDiagnosingMatcher Date: Sun, 5 Apr 2020 13:36:25 -0700 Subject: [PATCH 042/159] Quick fix --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 1 + .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 4 +--- 2 files changed, 2 insertions(+), 3 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 34c17bdf..387d9155 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 @@ -173,6 +173,7 @@ public class HierarchicalShardSyncer { } else { throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found."); } + newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); newLease.ownerSwitchesSinceCheckpoint(0L); return newLease; } 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 18a0af63..66ed7cbc 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 @@ -71,8 +71,6 @@ public class ShutdownTask implements ConsumerTask { @NonNull private final InitialPositionInStreamExtended initialPositionInStream; private final boolean cleanupLeasesOfCompletedShards; - private final boolean garbageCollectLeases = false; - private final boolean isLeaseTableEmpty = false; private final boolean ignoreUnexpectedChildShards; @NonNull private final LeaseCoordinator leaseCoordinator; @@ -174,7 +172,7 @@ public class ShutdownTask implements ConsumerTask { private void createLeasesForChildShardsIfNotExist() throws DependencyException, InvalidStateException, ProvisionedThroughputException { for(ChildShard childShard : childShards) { - if(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId()) == null) { + if(leaseCoordinator.getCurrentlyHeldLease(childShard.shardId()) == null) { final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard); leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); } From 5351c9ddabf55a231c0bdd1dfdf4c1e1b43247bc Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 25 Mar 2020 15:49:57 -0400 Subject: [PATCH 043/159] Adding hash range validation for list shards with filter. --- .../leases/HierarchicalShardSyncer.java | 77 ++++++++++++++++-- .../leases/HierarchicalShardSyncerTest.java | 79 ++++++++++++++++++- 2 files changed, 149 insertions(+), 7 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 ecd64952..d398fb34 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 @@ -68,6 +68,10 @@ public class HierarchicalShardSyncer { private final boolean isMultiStreamMode; + public static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); + public static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); + public static final int retriesForCompleteHashRange = 3; + private String streamIdentifier = ""; public HierarchicalShardSyncer() { @@ -340,12 +344,27 @@ public class HierarchicalShardSyncer { } private static List getShardListAtInitialPosition(@NonNull final ShardDetector shardDetector, - InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException { - final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); - final Optional> shards = Optional.of(shardDetector.listShardsWithFilter(shardFilter)); + InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException { - return shards.orElseThrow(() -> new KinesisClientLibIOException("Stream " + shardDetector.streamIdentifier().streamName() + - " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); + final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); + List shards; + + for (int i = 0; i < retriesForCompleteHashRange; i++) { + shards = shardDetector.listShardsWithFilter(shardFilter); + + if (shards == null) { + throw new KinesisClientLibIOException( + "Stream " + shardDetector.streamIdentifier().streamName() + + " is not in ACTIVE OR UPDATING state - will retry getting the shard list."); + } + + if (hashRangeOfShardsIsComplete(shards)) { + return shards; + } + } + + throw new KinesisClientLibIOException("Hash range of shards returned was incomplete after " + + retriesForCompleteHashRange + " retries."); } private static List getShardList(@NonNull final ShardDetector shardDetector) throws KinesisClientLibIOException { @@ -355,6 +374,30 @@ public class HierarchicalShardSyncer { " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); } + private static boolean hashRangeOfShardsIsComplete(@NonNull List shards) { + + final Comparator shardStartingHashKeyBasedComparator = new ShardStartingHashKeyBasedComparator(); + shards.sort(shardStartingHashKeyBasedComparator); + + if (!shards.get(0).hashKeyRange().startingHashKey().equals(MIN_HASH_KEY) || + !shards.get(shards.size() - 1).hashKeyRange().endingHashKey().equals(MAX_HASH_KEY)) { + return false; + } + + if (shards.size() > 1) { + for (int i = 1; i < shards.size(); i++) { + final BigInteger startOfPossibleHole = new BigInteger(shards.get(i - 1).hashKeyRange().endingHashKey()); + final BigInteger endOfPossibleHole = new BigInteger(shards.get(i).hashKeyRange().startingHashKey()); + + if (!endOfPossibleHole.subtract(startOfPossibleHole).equals(BigInteger.ONE)) { + return false; + } + } + } + + return true; + } + /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. @@ -414,7 +457,7 @@ public class HierarchicalShardSyncer { * Check if this shard is a descendant of a shard that is (or will be) processed. * Create leases for the ancestors of this shard as required. * See javadoc of determineNewLeasesToCreate() for rules and example. - * + * * @param shardId The shardId to check. * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * location in the shard (when an application starts up for the first time - and there are no checkpoints). @@ -789,6 +832,28 @@ public class HierarchicalShardSyncer { .map(streamId -> streamId.serialize()).orElse("single_stream_mode"); } + /** + * Helper class to compare shards based on their hash range. + */ + @RequiredArgsConstructor + private static class ShardStartingHashKeyBasedComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + + /** + * Compares two shards based on their starting hash keys. + * We assume that the shards provided are non-null. + * + * {@inheritDoc} + */ + @Override + public int compare(Shard shard1, Shard shard2) { + BigInteger hashKey1 = new BigInteger(shard1.hashKeyRange().startingHashKey()); + BigInteger hashKey2 = new BigInteger(shard2.hashKeyRange().startingHashKey()); + + return hashKey1.compareTo(hashKey2); + } + } + /** Helper class to compare leases based on starting sequence number of the corresponding shards. * */ 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 30e4f081..963f84fd 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 @@ -1683,7 +1683,84 @@ public class HierarchicalShardSyncerTest { verify(shardDetector, atLeast(1)).listShards(); } -// /**getShardFilterFromInitialPosition + /** + * Tries to boostrap empty lease table. Verifies that if we fail to get a complete hash range of shards after three + * retries, we fast fail and throw an exception. + * @throws Exception + */ + @Test(expected = KinesisClientLibIOException.class) + public void testEmptyLeaseTableThrowsExceptionWhenHashRangeIsStillIncompleteAfterRetries() throws Exception { + final List shardsWithIncompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("0", "1")), + ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("2", "3")) + ); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(any(ShardFilter.class))).thenReturn(shardsWithIncompleteHashRange); + + try { + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + } finally { + verify(shardDetector, times(3)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. + } + } + + /** + * Tries to bootstrap an empty lease table. Verifies that after getting an incomplete hash range of shards two times + * and a complete hash range the final time, we create the leases. + * @throws Exception + */ + @Test + public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRangeAfterTwoRetries() throws Exception { + final List shardsWithIncompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("0", "1")), + ShardObjectHelper.newShard("shardId-1", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("2", "3")) + ); + final List shardsWithCompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")), + ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY)) + ); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(any(ShardFilter.class))).thenReturn(shardsWithIncompleteHashRange) + .thenReturn(shardsWithIncompleteHashRange).thenReturn(shardsWithCompleteHashRange); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + verify(shardDetector, times(3)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. + verify(dynamoDBLeaseRefresher, times(2)).createLeaseIfNotExists(any(Lease.class)); + } + + /** + * Tries to bootstrap an empty lease table. Verifies that leases are created when we have a complete hash range of shards. + * @throws Exception + */ + @Test + public void testEmptyLeaseTablePopulatesLeasesWithCompleteHashRange() throws Exception { + final List shardsWithCompleteHashRange = Arrays.asList( + ShardObjectHelper.newShard("shardId-2", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, "420")), + ShardObjectHelper.newShard("shardId-3", null, null, ShardObjectHelper.newSequenceNumberRange("1", "2"), ShardObjectHelper.newHashKeyRange("421", ShardObjectHelper.MAX_HASH_KEY)) + ); + + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(true); + when(shardDetector.listShardsWithFilter(any(ShardFilter.class))).thenReturn(shardsWithCompleteHashRange); + + hierarchicalShardSyncer + .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, + SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); + + verify(shardDetector, times(1)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. + verify(dynamoDBLeaseRefresher, times(2)).createLeaseIfNotExists(any(Lease.class)); + } + +// /** // * Test CheckIfDescendantAndAddNewLeasesForAncestors - two parents, there is a lease for one parent. // */ // @Test From ea093a44660729e576b745591ebcde552ef3c98d Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 30 Mar 2020 19:30:11 -0400 Subject: [PATCH 044/159] Adding stream id to exception --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 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 d398fb34..07521335 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 @@ -347,6 +347,8 @@ public class HierarchicalShardSyncer { InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException { final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); + final String streamName = shardDetector.streamIdentifier().streamName(); + List shards; for (int i = 0; i < retriesForCompleteHashRange; i++) { @@ -354,8 +356,7 @@ public class HierarchicalShardSyncer { if (shards == null) { throw new KinesisClientLibIOException( - "Stream " + shardDetector.streamIdentifier().streamName() + - " is not in ACTIVE OR UPDATING state - will retry getting the shard list."); + "Stream " + streamName + " is not in ACTIVE OR UPDATING state - will retry getting the shard list."); } if (hashRangeOfShardsIsComplete(shards)) { @@ -363,7 +364,7 @@ public class HierarchicalShardSyncer { } } - throw new KinesisClientLibIOException("Hash range of shards returned was incomplete after " + throw new KinesisClientLibIOException("Hash range of shards returned for " + streamName + " was incomplete after " + retriesForCompleteHashRange + " retries."); } From 01db7753f0e37a62c4c587d56415728812c74866 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 8 Apr 2020 00:31:49 -0400 Subject: [PATCH 045/159] Making test cases actually go through validation steps. --- .../amazon/kinesis/leases/HierarchicalShardSyncerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 963f84fd..e1dfc52a 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 @@ -1716,8 +1716,8 @@ 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("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(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")), From c7fe06dab50112f9223e179020fd2e538965df62 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 8 Apr 2020 00:31:55 -0400 Subject: [PATCH 046/159] PR comments --- .../leases/HierarchicalShardSyncer.java | 27 +++++++++++++------ .../leases/exceptions/ShardSyncer.java | 6 ++--- 2 files changed, 22 insertions(+), 11 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 07521335..e0acf862 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 @@ -68,12 +68,14 @@ public class HierarchicalShardSyncer { private final boolean isMultiStreamMode; - public static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); - public static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); - public static final int retriesForCompleteHashRange = 3; - private String streamIdentifier = ""; + private static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); + private static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); + private static final int retriesForCompleteHashRange = 3; + + private static final long DELAY_BETWEEN_LIST_SHARDS_MILLIS = 1000; + public HierarchicalShardSyncer() { isMultiStreamMode = false; } @@ -109,7 +111,7 @@ public class HierarchicalShardSyncer { final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final MetricsScope scope, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) - throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { this.streamIdentifier = shardDetector.streamIdentifier().serialize(); final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); @@ -344,7 +346,7 @@ public class HierarchicalShardSyncer { } private static List getShardListAtInitialPosition(@NonNull final ShardDetector shardDetector, - InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException { + InitialPositionInStreamExtended initialPositionInStreamExtended) throws KinesisClientLibIOException, InterruptedException { final ShardFilter shardFilter = getShardFilterFromInitialPosition(initialPositionInStreamExtended); final String streamName = shardDetector.streamIdentifier().streamName(); @@ -362,6 +364,8 @@ public class HierarchicalShardSyncer { if (hashRangeOfShardsIsComplete(shards)) { return shards; } + + Thread.sleep(DELAY_BETWEEN_LIST_SHARDS_MILLIS); } throw new KinesisClientLibIOException("Hash range of shards returned for " + streamName + " was incomplete after " @@ -377,6 +381,10 @@ public class HierarchicalShardSyncer { private static boolean hashRangeOfShardsIsComplete(@NonNull List shards) { + if (shards.isEmpty()) { + throw new IllegalStateException("No shards found when attempting to validate complete hash range."); + } + final Comparator shardStartingHashKeyBasedComparator = new ShardStartingHashKeyBasedComparator(); shards.sort(shardStartingHashKeyBasedComparator); @@ -387,10 +395,13 @@ public class HierarchicalShardSyncer { if (shards.size() > 1) { for (int i = 1; i < shards.size(); i++) { - final BigInteger startOfPossibleHole = new BigInteger(shards.get(i - 1).hashKeyRange().endingHashKey()); - final BigInteger endOfPossibleHole = new BigInteger(shards.get(i).hashKeyRange().startingHashKey()); + final Shard shardAtStartOfPossibleHole = shards.get(i - 1); + final Shard shardAtEndOfPossibleHole = shards.get(i); + final BigInteger startOfPossibleHole = new BigInteger(shardAtStartOfPossibleHole.hashKeyRange().endingHashKey()); + final BigInteger endOfPossibleHole = new BigInteger(shardAtEndOfPossibleHole.hashKeyRange().startingHashKey()); if (!endOfPossibleHole.subtract(startOfPossibleHole).equals(BigInteger.ONE)) { + log.error("Incomplete hash range found between {} and {}.", shardAtStartOfPossibleHole, shardAtEndOfPossibleHole); return false; } } 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 c0ed2d2a..182854ff 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 @@ -40,8 +40,8 @@ public class ShardSyncer { final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, - KinesisClientLibIOException { - HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - scope, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, leaseRefresher.isLeaseTableEmpty()); + KinesisClientLibIOException, InterruptedException { + HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, + scope, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, leaseRefresher.isLeaseTableEmpty()); } } From 0715903456ab5d9e87a56906ba6d95106df2022f Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 8 Apr 2020 16:47:37 -0700 Subject: [PATCH 047/159] Addressing comments --- .../leases/HierarchicalShardSyncer.java | 48 ++++++++++---- .../CustomerApplicationException.java | 10 +++ .../kinesis/lifecycle/ShutdownTask.java | 63 ++++++++++--------- .../retrieval/polling/KinesisDataFetcher.java | 23 ++++--- .../kinesis/lifecycle/ConsumerStatesTest.java | 3 - .../kinesis/lifecycle/ShutdownTaskTest.java | 3 +- 6 files changed, 90 insertions(+), 60 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java 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 387d9155..3782d177 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 @@ -165,19 +165,6 @@ public class HierarchicalShardSyncer { } } - public synchronized Lease createLeaseForChildShard(ChildShard childShard) throws InvalidStateException { - Lease newLease = new Lease(); - newLease.leaseKey(childShard.shardId()); - if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) { - newLease.parentShardIds(childShard.parentShards()); - } else { - throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found."); - } - newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); - newLease.ownerSwitchesSinceCheckpoint(0L); - return newLease; - } - // CHECKSTYLE:ON CyclomaticComplexity /** Note: This method has package level access solely for testing purposes. @@ -722,6 +709,41 @@ public class HierarchicalShardSyncer { } } + 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) + : newKCLLeaseForChildShard(childShard); + } + + private static Lease newKCLLeaseForChildShard(final ChildShard childShard) throws InvalidStateException { + Lease newLease = new Lease(); + newLease.leaseKey(childShard.shardId()); + if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + newLease.parentShardIds(childShard.parentShards()); + } else { + 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); + return newLease; + } + + private static Lease newKCLMultiStreamLeaseForChildShard(final ChildShard childShard, final StreamIdentifier streamIdentifier) throws InvalidStateException { + MultiStreamLease newLease = new MultiStreamLease(); + newLease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), childShard.shardId())); + if (!CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + newLease.parentShardIds(childShard.parentShards()); + } else { + throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.shardId() + "because parent shards cannot be found."); + } + newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.streamIdentifier(streamIdentifier.serialize()); + newLease.shardId(childShard.shardId()); + return newLease; + } + /** * Helper method to create a new Lease POJO for a shard. * Note: Package level access only for testing purposes diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java new file mode 100644 index 00000000..40121e98 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/CustomerApplicationException.java @@ -0,0 +1,10 @@ +package software.amazon.kinesis.leases.exceptions; + +public class CustomerApplicationException extends Exception { + + public CustomerApplicationException(Throwable e) { super(e);} + + public CustomerApplicationException(String message, Throwable e) { super(message, e);} + + public CustomerApplicationException(String message) { super(message);} +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 66ed7cbc..95e52068 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 @@ -31,11 +31,11 @@ import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.HierarchicalShardSyncer; +import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; 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.LeaseLostInput; -import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsScope; @@ -100,7 +100,6 @@ public class ShutdownTask implements ConsumerTask { final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, SHUTDOWN_TASK_OPERATION); Exception exception; - boolean applicationException = false; try { try { @@ -117,49 +116,30 @@ public class ShutdownTask implements ConsumerTask { recordProcessorCheckpointer .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); - // Call the shardReocrdsProcessor 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. - try { - shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); - if (lastCheckpointValue == null - || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { - throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfoIdProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + - "See ShardRecordProcessor.shardEnded javadocs for more information."); - } - } catch (Exception e) { - applicationException = true; - throw e; - } finally { - MetricsUtil.addLatency(scope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); - } + // 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(() -> applicationCheckpointWithShardEnd(), scope, startTime); } else { - try { - shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()); - } catch (Exception e) { - applicationException = true; - throw e; - } + throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); } - log.debug("Shutting down retrieval strategy."); + log.debug("Shutting down retrieval strategy for shard {}.", shardInfoIdProvider.apply(shardInfo)); recordsPublisher.shutdown(); log.debug("Record processor completed shutdown() for shard {}", shardInfoIdProvider.apply(shardInfo)); return new TaskResult(null); } catch (Exception e) { - if (applicationException) { - log.error("Application exception. ", e); + if (e instanceof CustomerApplicationException) { + log.error("Shard {}: Application exception. ", shardInfoIdProvider.apply(shardInfo), e); } else { - log.error("Caught exception: ", e); + log.error("Shard {}: Caught exception: ", shardInfoIdProvider.apply(shardInfo), e); } exception = e; // backoff if we encounter an exception. try { Thread.sleep(this.backoffTimeMillis); } catch (InterruptedException ie) { - log.debug("Interrupted sleep", ie); + log.debug("Shard{}: Interrupted sleep", shardInfoIdProvider.apply(shardInfo), ie); } } } finally { @@ -169,11 +149,32 @@ public class ShutdownTask implements ConsumerTask { return new TaskResult(exception); } + private void applicationCheckpointWithShardEnd() { + shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); + final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); + if (lastCheckpointValue == null + || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { + throw new IllegalArgumentException("Application didn't checkpoint at end of shard " + + shardInfoIdProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + + "See ShardRecordProcessor.shardEnded javadocs for more information."); + } + } + + private void throwOnApplicationException(Runnable action, MetricsScope metricsScope, final long startTime) throws CustomerApplicationException { + try { + action.run(); + } catch (Exception e) { + throw new CustomerApplicationException("Customer application throws exception for shard " + shardInfoIdProvider.apply(shardInfo) +": ", e); + } finally { + MetricsUtil.addLatency(metricsScope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); + } + } + private void createLeasesForChildShardsIfNotExist() throws DependencyException, InvalidStateException, ProvisionedThroughputException { for(ChildShard childShard : childShards) { if(leaseCoordinator.getCurrentlyHeldLease(childShard.shardId()) == null) { - final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard); + final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); } } 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 dc25b20c..a96e2134 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 @@ -121,13 +121,7 @@ public class KinesisDataFetcher { if (nextIterator != null) { try { - GetRecordsResponse getRecordsResponse = getRecords(nextIterator); - while (!isValidResponse(getRecordsResponse)) { - log.error("{} : GetRecords response is not valid. nextShardIterator: {}. childShards: {}. Will retry GetRecords with the same nextIterator.", - shardId, getRecordsResponse.nextShardIterator(), getRecordsResponse.childShards()); - getRecordsResponse = getRecords(nextIterator); - } - return new AdvancingResult(getRecordsResponse); + return new AdvancingResult(getRecords(nextIterator)); } catch (ResourceNotFoundException e) { log.info("Caught ResourceNotFoundException when fetching records for shard {}", streamAndShardId); return TERMINAL_RESULT; @@ -188,11 +182,6 @@ public class KinesisDataFetcher { } } - private boolean isValidResponse(GetRecordsResponse response) { - return response.nextShardIterator() == null ? !CollectionUtils.isNullOrEmpty(response.childShards()) - : response.childShards() != null && response.childShards().isEmpty(); - } - /** * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number. * @param initialCheckpoint Current checkpoint sequence number for this shard. @@ -297,6 +286,11 @@ public class KinesisDataFetcher { try { final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request), maxFutureWait); + if (!isValidResponse(response)) { + throw new RetryableRetrievalException("GetRecords response is not valid for shard: " + streamAndShardId + + ". nextShardIterator: " + response.nextShardIterator() + + ". childShards: " + response.childShards() + ". Will retry GetRecords with the same nextIterator."); + } success = true; return response; } catch (ExecutionException e) { @@ -314,6 +308,11 @@ public class KinesisDataFetcher { } } + private boolean isValidResponse(GetRecordsResponse response) { + return response.nextShardIterator() == null ? !CollectionUtils.isNullOrEmpty(response.childShards()) + : response.childShards() != null && response.childShards().isEmpty(); + } + private AWSExceptionManager createExceptionManager() { final AWSExceptionManager exceptionManager = new AWSExceptionManager(); exceptionManager.add(ResourceNotFoundException.class, t -> t); 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 b4164f90..06e5afc7 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 @@ -35,7 +35,6 @@ import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -61,8 +60,6 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; -import javax.swing.*; -import javax.swing.text.AsyncBoxView; @RunWith(MockitoJUnitRunner.class) public class ConsumerStatesTest { 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 8a9024f6..cbb9b834 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 @@ -52,6 +52,7 @@ import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardObjectHelper; +import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -125,7 +126,7 @@ public class ShutdownTaskTest { final TaskResult result = task.call(); assertNotNull(result.getException()); - assertTrue(result.getException() instanceof IllegalArgumentException); + assertTrue(result.getException() instanceof CustomerApplicationException); } /** From 734d4918d0be4dad61253d5ef7fbc92b6c409f35 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 9 Apr 2020 10:48:19 -0700 Subject: [PATCH 048/159] Addressing minor comments --- .../CustomerApplicationException.java | 17 +++++++++++++++++ .../amazon/kinesis/lifecycle/ShutdownTask.java | 8 +++++--- 2 files changed, 22 insertions(+), 3 deletions(-) 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 40121e98..ba97ab08 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 @@ -1,5 +1,22 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package software.amazon.kinesis.leases.exceptions; +/** + * Exception type for all exceptions thrown by the customer implemented code. + */ public class CustomerApplicationException extends Exception { public CustomerApplicationException(Throwable e) { super(e);} 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 95e52068..33eb4497 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 @@ -111,6 +111,8 @@ public class ShutdownTask implements ConsumerTask { // Create new lease for the child shards if they don't exist. if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); + } else { + log.warn("Shard {} no longer exists. Shutting down consumer with SHARD_END reason without creating leases for child shards.", shardInfoIdProvider.apply(shardInfo)); } recordProcessorCheckpointer @@ -118,7 +120,7 @@ public class ShutdownTask implements ConsumerTask { recordProcessorCheckpointer.largestPermittedCheckpointValue(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(() -> applicationCheckpointWithShardEnd(), scope, startTime); + throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime); } else { throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); } @@ -139,7 +141,7 @@ public class ShutdownTask implements ConsumerTask { try { Thread.sleep(this.backoffTimeMillis); } catch (InterruptedException ie) { - log.debug("Shard{}: Interrupted sleep", shardInfoIdProvider.apply(shardInfo), ie); + log.debug("Shard {}: Interrupted sleep", shardInfoIdProvider.apply(shardInfo), ie); } } } finally { @@ -149,7 +151,7 @@ public class ShutdownTask implements ConsumerTask { return new TaskResult(exception); } - private void applicationCheckpointWithShardEnd() { + private void applicationCheckpointAndVerification() { shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); if (lastCheckpointValue == null From af0cd5463df82e85a6de9031ddfd4be82a27717b Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 30 Mar 2020 02:35:13 -0400 Subject: [PATCH 049/159] Adding arbitrary application state checkpointing for two phase commit. --- .../StreamingShardRecordProcessorTest.java | 20 ++++++++++ .../amazon/kinesis/checkpoint/Checkpoint.java | 7 +++- .../ShardRecordProcessorCheckpointer.java | 40 ++++++++++++++----- .../dynamodb/DynamoDBCheckpointer.java | 12 ++++-- .../software/amazon/kinesis/leases/Lease.java | 15 +++++++ .../kinesis/processor/Checkpointer.java | 5 ++- .../RecordProcessorCheckpointer.java | 16 +++++++- .../checkpoint/InMemoryCheckpointer.java | 9 ++++- .../kinesis/coordinator/SchedulerTest.java | 4 +- 9 files changed, 110 insertions(+), 18 deletions(-) diff --git a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java index da7e9fb2..e3368e07 100644 --- a/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java +++ b/amazon-kinesis-client-multilang/src/test/java/software/amazon/kinesis/multilang/StreamingShardRecordProcessorTest.java @@ -112,6 +112,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + throw new UnsupportedOperationException(); + } + @Override public PreparedCheckpointer prepareCheckpoint(Record record) throws KinesisClientLibDependencyException, @@ -119,6 +124,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + throw new UnsupportedOperationException(); + } + @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber) throws KinesisClientLibDependencyException, @@ -126,6 +136,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + return null; + } + @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) throws KinesisClientLibDependencyException, @@ -133,6 +148,11 @@ public class StreamingShardRecordProcessorTest { throw new UnsupportedOperationException(); } + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + throw new UnsupportedOperationException(); + } + @Override public Checkpointer checkpointer() { throw new UnsupportedOperationException(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java index 2bab0cd6..7b7bea9e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java @@ -18,6 +18,8 @@ import lombok.Data; import lombok.experimental.Accessors; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.util.Arrays; + /** * A class encapsulating the 2 pieces of state stored in a checkpoint. */ @@ -26,18 +28,21 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; public class Checkpoint { private final ExtendedSequenceNumber checkpoint; private final ExtendedSequenceNumber pendingCheckpoint; + private final byte[] pendingCheckpointState; /** * Constructor. * * @param checkpoint the checkpoint sequence number - cannot be null or empty. * @param pendingCheckpoint the pending checkpoint sequence number - can be null. + * @param pendingCheckpointState the pending checkpoint state - can be null. */ - public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) { + public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState) { if (checkpoint == null || checkpoint.sequenceNumber().isEmpty()) { throw new IllegalArgumentException("Checkpoint cannot be null or empty"); } this.checkpoint = checkpoint; this.pendingCheckpoint = pendingCheckpoint; + this.pendingCheckpointState = pendingCheckpointState; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java index 7d504bbb..0f823915 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 @@ -140,12 +140,13 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi this.largestPermittedCheckpointValue.subSequenceNumber()); } - /** - * {@inheritDoc} - */ @Override - public synchronized PreparedCheckpointer prepareCheckpoint(Record record) - throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + return prepareCheckpoint(largestPermittedCheckpointValue.sequenceNumber(), applicationState); + } + + @Override + public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { // // TODO: UserRecord Deprecation // @@ -154,10 +155,19 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } /*else if (record instanceof UserRecord) { return prepareCheckpoint(record.sequenceNumber(), ((UserRecord) record).subSequenceNumber()); } */ else { - return prepareCheckpoint(record.sequenceNumber(), 0); + return prepareCheckpoint(record.sequenceNumber(), 0, applicationState); } } + /** + * {@inheritDoc} + */ + @Override + public synchronized PreparedCheckpointer prepareCheckpoint(Record record) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + return prepareCheckpoint(record, null); + } + /** * {@inheritDoc} */ @@ -167,13 +177,24 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi return prepareCheckpoint(sequenceNumber, 0); } + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + return prepareCheckpoint(sequenceNumber, 0, null); + } + /** * {@inheritDoc} */ @Override public synchronized PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + return prepareCheckpoint(sequenceNumber, subSequenceNumber, null); + } + @Override + public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { if (subSequenceNumber < 0) { throw new IllegalArgumentException("Could not checkpoint at invalid, negative subsequence number " + subSequenceNumber); @@ -191,7 +212,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi log.debug("Preparing checkpoint {}, token {} at specific extended sequence number {}", ShardInfo.getLeaseKey(shardInfo), shardInfo.concurrencyToken(), pendingCheckpoint); } - return doPrepareCheckpoint(pendingCheckpoint); + return doPrepareCheckpoint(pendingCheckpoint, applicationState); } else { throw new IllegalArgumentException(String.format( "Could not prepare checkpoint at extended sequence number %s as it did not fall into acceptable " @@ -290,7 +311,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi * @throws ThrottlingException * @throws ShutdownException */ - private PreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber) + private PreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { ExtendedSequenceNumber newPrepareCheckpoint = extendedSequenceNumber; @@ -308,7 +329,8 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } try { - checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken()); + checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, applicationState, + shardInfo.concurrencyToken()); } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { throw e; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java index fb7a6fc7..71969f65 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 @@ -88,7 +88,7 @@ public class DynamoDBCheckpointer implements Checkpointer { try { Lease lease = leaseRefresher.getLease(leaseKey); log.debug("[{}] Retrieved lease => {}", leaseKey, lease); - return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint()); + return new Checkpoint(lease.checkpoint(), lease.pendingCheckpoint(), lease.pendingCheckpointState()); } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { String message = "Unable to fetch checkpoint for shardId " + leaseKey; log.error(message, e); @@ -99,9 +99,14 @@ public class DynamoDBCheckpointer implements Checkpointer { @Override public void prepareCheckpoint(final String leaseKey, final ExtendedSequenceNumber pendingCheckpoint, final String concurrencyToken) throws KinesisClientLibException { + prepareCheckpoint(leaseKey, pendingCheckpoint, null, concurrencyToken); + } + + @Override + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) throws KinesisClientLibException { try { boolean wasSuccessful = - prepareCheckpoint(leaseKey, pendingCheckpoint, UUID.fromString(concurrencyToken)); + prepareCheckpoint(leaseKey, pendingCheckpoint, pendingCheckpointState, UUID.fromString(concurrencyToken)); if (!wasSuccessful) { throw new ShutdownException( "Can't prepare checkpoint - instance doesn't hold the lease for this shard"); @@ -134,7 +139,7 @@ public class DynamoDBCheckpointer implements Checkpointer { return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } - boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken) + boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, UUID concurrencyToken) throws DependencyException, InvalidStateException, ProvisionedThroughputException { Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey); if (lease == null) { @@ -144,6 +149,7 @@ public class DynamoDBCheckpointer implements Checkpointer { } lease.pendingCheckpoint(Objects.requireNonNull(pendingCheckpoint, "pendingCheckpoint should not be null")); + lease.pendingCheckpointState(pendingCheckpointState); return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } 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 682a6f9e..efac63b0 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 @@ -82,6 +82,12 @@ public class Lease { * @return pending checkpoint, possibly null. */ private ExtendedSequenceNumber pendingCheckpoint; + + /** + * @return pending checkpoint state, possibly null. + */ + private byte[] pendingCheckpointState; + /** * @return count of distinct lease holders between checkpoints. */ @@ -212,6 +218,15 @@ public class Lease { this.pendingCheckpoint = pendingCheckpoint; } + /** + * Sets pending checkpoint state. + * + * @param pendingCheckpointState can be null + */ + public void pendingCheckpointState(byte[] pendingCheckpointState) { + this.pendingCheckpointState = pendingCheckpointState; + } + /** * Sets ownerSwitchesSinceCheckpoint. * diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java index 70cdd608..ad44df3e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java @@ -40,7 +40,7 @@ public interface Checkpointer { /** * Get the current checkpoint stored for the specified shard. Useful for checking that the parent shard * has been completely processed before we start processing the child shard. - * + * * @param leaseKey Current checkpoint for this shard is fetched * @return Current checkpoint for this shard, null if there is no record for this shard. * @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint @@ -73,6 +73,9 @@ public interface Checkpointer { void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException; + void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) + throws KinesisClientLibException; + void operation(String operation); String operation(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java index 2eb3f5c1..d42cbb9e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java @@ -93,7 +93,6 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; - /** * This method will checkpoint the progress at the provided sequenceNumber and subSequenceNumber, the latter for * aggregated records produced with the Producer Library. This method is analogous to {@link #checkpoint()} @@ -145,6 +144,9 @@ public interface RecordProcessorCheckpointer { PreparedCheckpointer prepareCheckpoint() throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + PreparedCheckpointer prepareCheckpoint(byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** * This method will record a pending checkpoint at the at the provided record. This method is analogous to * {@link #prepareCheckpoint()} but provides the ability to specify the record at which to prepare the checkpoint. @@ -174,6 +176,10 @@ public interface RecordProcessorCheckpointer { PreparedCheckpointer prepareCheckpoint(Record record) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + + PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** * This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to * {@link #prepareCheckpoint()} but provides the ability to specify the sequence number at which to checkpoint. @@ -200,6 +206,10 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; + /** * This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for * aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()} @@ -228,5 +238,9 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; + Checkpointer checkpointer(); } 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 ebe933b9..83327931 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 @@ -32,6 +32,7 @@ public class InMemoryCheckpointer implements Checkpointer { private Map checkpoints = new HashMap<>(); private Map flushpoints = new HashMap<>(); private Map pendingCheckpoints = new HashMap<>(); + private Map pendingCheckpointStates = new HashMap<>(); private String operation; @@ -64,6 +65,11 @@ public class InMemoryCheckpointer implements Checkpointer { @Override public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException { + prepareCheckpoint(leaseKey, pendingCheckpoint, null, concurrencyToken); + } + + @Override + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) throws KinesisClientLibException { pendingCheckpoints.put(leaseKey, pendingCheckpoint); } @@ -71,8 +77,9 @@ public class InMemoryCheckpointer implements Checkpointer { public Checkpoint getCheckpointObject(String leaseKey) throws KinesisClientLibException { ExtendedSequenceNumber checkpoint = flushpoints.get(leaseKey); ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(leaseKey); + byte[] pendingCheckpointState = pendingCheckpointStates.get(leaseKey); - Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint); + Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint, pendingCheckpointState); log.debug("getCheckpointObject shardId: {}, {}", leaseKey, checkpointObj); return checkpointObj; } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java index fd6b531b..1de7b101 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 @@ -242,7 +242,7 @@ public class SchedulerTest { final List secondShardInfo = Collections.singletonList( new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber)); - final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null); + final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null, null); when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo); when(checkpoint.getCheckpointObject(eq(shardId))).thenReturn(firstCheckpoint); @@ -368,7 +368,7 @@ public class SchedulerTest { .map(sc -> new ShardInfo(shardId, concurrencyToken, null, finalSequenceNumber, sc.streamIdentifier().serialize())).collect(Collectors.toList()); - final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null); + final Checkpoint firstCheckpoint = new Checkpoint(firstSequenceNumber, null, null); when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialShardInfo, firstShardInfo, secondShardInfo); when(checkpoint.getCheckpointObject(anyString())).thenReturn(firstCheckpoint); From 26c737cc2a3a36e27913274e3139dc8243cd747b Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 03:22:56 -0400 Subject: [PATCH 050/159] Adding in memory implememtation for pending checkpoint state --- .../software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java | 1 + 1 file changed, 1 insertion(+) 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 83327931..a6190dfc 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 @@ -71,6 +71,7 @@ public class InMemoryCheckpointer implements Checkpointer { @Override public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) throws KinesisClientLibException { pendingCheckpoints.put(leaseKey, pendingCheckpoint); + pendingCheckpointStates.put(leaseKey, pendingCheckpointState); } @Override From 5355b4b7c5d7910c6c9de0bff5b5111cad1f44a9 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 03:28:58 -0400 Subject: [PATCH 051/159] Moving new parameters to end --- .../checkpoint/ShardRecordProcessorCheckpointer.java | 4 ++-- .../kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java | 8 ++++---- .../software/amazon/kinesis/processor/Checkpointer.java | 2 +- .../amazon/kinesis/checkpoint/InMemoryCheckpointer.java | 5 ++--- 4 files changed, 9 insertions(+), 10 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 0f823915..ec9ff11d 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 @@ -329,8 +329,8 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } try { - checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, applicationState, - shardInfo.concurrencyToken()); + checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken(), applicationState + ); } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { throw e; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java index 71969f65..b93c3779 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 @@ -99,14 +99,14 @@ public class DynamoDBCheckpointer implements Checkpointer { @Override public void prepareCheckpoint(final String leaseKey, final ExtendedSequenceNumber pendingCheckpoint, final String concurrencyToken) throws KinesisClientLibException { - prepareCheckpoint(leaseKey, pendingCheckpoint, null, concurrencyToken); + prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null); } @Override - public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) throws KinesisClientLibException { + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException { try { boolean wasSuccessful = - prepareCheckpoint(leaseKey, pendingCheckpoint, pendingCheckpointState, UUID.fromString(concurrencyToken)); + prepareCheckpoint(leaseKey, pendingCheckpoint, UUID.fromString(concurrencyToken), pendingCheckpointState); if (!wasSuccessful) { throw new ShutdownException( "Can't prepare checkpoint - instance doesn't hold the lease for this shard"); @@ -139,7 +139,7 @@ public class DynamoDBCheckpointer implements Checkpointer { return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); } - boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, UUID concurrencyToken) + boolean prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken, byte[] pendingCheckpointState) throws DependencyException, InvalidStateException, ProvisionedThroughputException { Lease lease = leaseCoordinator.getCurrentlyHeldLease(leaseKey); if (lease == null) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java index ad44df3e..7f3c947e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java @@ -73,7 +73,7 @@ public interface Checkpointer { void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException; - void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) + void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException; void operation(String operation); 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 a6190dfc..661cf2ff 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java @@ -18,7 +18,6 @@ import java.util.HashMap; import java.util.Map; import software.amazon.kinesis.exceptions.KinesisClientLibException; -import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.processor.Checkpointer; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @@ -65,11 +64,11 @@ public class InMemoryCheckpointer implements Checkpointer { @Override public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException { - prepareCheckpoint(leaseKey, pendingCheckpoint, null, concurrencyToken); + prepareCheckpoint(leaseKey, pendingCheckpoint, concurrencyToken, null); } @Override - public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, byte[] pendingCheckpointState, String concurrencyToken) throws KinesisClientLibException { + public void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException { pendingCheckpoints.put(leaseKey, pendingCheckpoint); pendingCheckpointStates.put(leaseKey, pendingCheckpointState); } From 1794874e3365d38ea2cc2c83bed1c3ff844dc9e9 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 03:45:03 -0400 Subject: [PATCH 052/159] Clean up checkpoint state after successful checkpoint --- .../amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java | 1 + .../software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java | 1 + 2 files changed, 2 insertions(+) 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 b93c3779..d9646351 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/dynamodb/DynamoDBCheckpointer.java @@ -134,6 +134,7 @@ public class DynamoDBCheckpointer implements Checkpointer { lease.checkpoint(checkpoint); lease.pendingCheckpoint(null); + lease.pendingCheckpointState(null); lease.ownerSwitchesSinceCheckpoint(0L); return leaseCoordinator.updateLease(lease, concurrencyToken, operation, leaseKey); 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 661cf2ff..8f6e165d 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/InMemoryCheckpointer.java @@ -44,6 +44,7 @@ public class InMemoryCheckpointer implements Checkpointer { checkpoints.put(leaseKey, checkpointValue); flushpoints.put(leaseKey, checkpointValue); pendingCheckpoints.remove(leaseKey); + pendingCheckpointStates.remove(leaseKey); if (log.isDebugEnabled()) { log.debug("shardId: {} checkpoint: {}", leaseKey, checkpointValue); From ae005ce0f898992d36855b5d3fe5737a5449d2e3 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 04:03:38 -0400 Subject: [PATCH 053/159] Adding unit tests --- .../kinesis/checkpoint/CheckpointerTest.java | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java index 1cf77a3d..b823c8e3 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/checkpoint/CheckpointerTest.java @@ -90,6 +90,26 @@ public class CheckpointerTest { Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); } + + @Test + public final void testInitialPrepareCheckpointWithApplicationState() throws Exception { + String sequenceNumber = "1"; + String pendingCheckpointValue = "99999"; + String shardId = "myShardId"; + byte[] applicationState = "applicationState".getBytes(); + ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(sequenceNumber); + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken); + + ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue); + checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), testConcurrencyToken, + applicationState); + + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + } + @Test public final void testAdvancingPrepareCheckpoint() throws Exception { String shardId = "myShardId"; @@ -107,6 +127,26 @@ public class CheckpointerTest { } } + @Test + public final void testAdvancingPrepareCheckpointWithApplicationState() throws Exception { + String shardId = "myShardId"; + String checkpointValue = "12345"; + byte[] applicationState = "applicationState".getBytes(); + ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue); + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), testConcurrencyToken); + + for (Integer i = 0; i < 10; i++) { + String sequenceNumber = i.toString(); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber); + checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken, + applicationState); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + } + } + @Test public final void testPrepareAndSetCheckpoint() throws Exception { String checkpointValue = "12345"; @@ -134,4 +174,35 @@ public class CheckpointerTest { Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); } + + @Test + public final void testPrepareAndSetCheckpointWithApplicationState() throws Exception { + String checkpointValue = "12345"; + String shardId = "testShardId-1"; + String concurrencyToken = "token-1"; + String pendingCheckpointValue = "99999"; + byte[] applicationState = "applicationState".getBytes(); + + // set initial checkpoint + ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue); + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), concurrencyToken); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + + // prepare checkpoint + ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue); + checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken, applicationState); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(applicationState, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + + // do checkpoint + checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).checkpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).pendingCheckpointState()); + } } From b335246a306db03b55d9287dbb2522d3e0c11169 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 06:15:18 -0400 Subject: [PATCH 054/159] Adding serializer for application state to lease info --- .../amazon/kinesis/leases/DynamoUtils.java | 18 ++++++++++++++++++ .../dynamodb/DynamoDBLeaseSerializer.java | 15 +++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java index 9d5f9ae2..29d6029b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/DynamoUtils.java @@ -14,6 +14,7 @@ */ package software.amazon.kinesis.leases; +import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -36,6 +37,14 @@ public class DynamoUtils { return AttributeValue.builder().ss(collectionValue).build(); } + public static AttributeValue createAttributeValue(byte[] byteBufferValue) { + if (byteBufferValue == null) { + throw new IllegalArgumentException("Byte buffer attributeValues cannot be null or empty."); + } + + return AttributeValue.builder().b(SdkBytes.fromByteArray(byteBufferValue)).build(); + } + public static AttributeValue createAttributeValue(String stringValue) { if (stringValue == null || stringValue.isEmpty()) { throw new IllegalArgumentException("String attributeValues cannot be null or empty."); @@ -52,6 +61,15 @@ public class DynamoUtils { return AttributeValue.builder().n(longValue.toString()).build(); } + public static byte[] safeGetByteArray(Map dynamoRecord, String key) { + AttributeValue av = dynamoRecord.get(key); + if (av == null) { + return null; + } else { + return av.b().asByteArray(); + } + } + public static Long safeGetLong(Map dynamoRecord, String key) { AttributeValue av = dynamoRecord.get(key); if (av == null) { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java index a02e2a6e..f42bafcf 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 @@ -50,6 +50,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String CHECKPOINT_SUBSEQUENCE_NUMBER_KEY = "checkpointSubSequenceNumber"; private static final String PENDING_CHECKPOINT_SEQUENCE_KEY = "pendingCheckpoint"; private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber"; + private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; @Override @@ -75,6 +76,10 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().subSequenceNumber())); } + if (lease.pendingCheckpointState() != null) { + result.put(PENDING_CHECKPOINT_STATE_KEY, DynamoUtils.createAttributeValue(lease.checkpoint().subSequenceNumber())); + } + return result; } @@ -105,6 +110,9 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { DynamoUtils.safeGetLong(dynamoRecord, PENDING_CHECKPOINT_SUBSEQUENCE_KEY)) ); } + + leaseToUpdate.pendingCheckpointState(DynamoUtils.safeGetByteArray(dynamoRecord, PENDING_CHECKPOINT_STATE_KEY)); + return leaseToUpdate; } @@ -220,6 +228,13 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); } + + if (lease.pendingCheckpointState() != null) { + result.put(PENDING_CHECKPOINT_STATE_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.pendingCheckpointState()))); + } else { + result.put(PENDING_CHECKPOINT_STATE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); + } + return result; } From ddaf714a09c7cafc31c3dddea003f973054f3efc Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 06:15:47 -0400 Subject: [PATCH 055/159] Adding application state to lease copy --- .../src/main/java/software/amazon/kinesis/leases/Lease.java | 6 ++++-- .../amazon/kinesis/leases/HierarchicalShardSyncerTest.java | 2 +- .../java/software/amazon/kinesis/leases/LeaseBuilder.java | 3 ++- .../kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java | 2 +- 4 files changed, 8 insertions(+), 5 deletions(-) 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 efac63b0..a7ed666c 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 @@ -102,13 +102,13 @@ public class Lease { protected Lease(Lease lease) { this(lease.leaseKey(), lease.leaseOwner(), lease.leaseCounter(), lease.concurrencyToken(), lease.lastCounterIncrementNanos(), lease.checkpoint(), lease.pendingCheckpoint(), - lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds()); + lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.pendingCheckpointState()); } public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, final UUID concurrencyToken, final Long lastCounterIncrementNanos, final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, - final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds) { + final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds, final byte[] pendingCheckpointState) { this.leaseKey = leaseKey; this.leaseOwner = leaseOwner; this.leaseCounter = leaseCounter; @@ -120,6 +120,7 @@ public class Lease { if (parentShardIds != null) { this.parentShardIds.addAll(parentShardIds); } + this.pendingCheckpointState = pendingCheckpointState; } /** @@ -139,6 +140,7 @@ public class Lease { ownerSwitchesSinceCheckpoint(lease.ownerSwitchesSinceCheckpoint()); checkpoint(lease.checkpoint); pendingCheckpoint(lease.pendingCheckpoint); + pendingCheckpointState(lease.pendingCheckpointState); parentShardIds(lease.parentShardIds); } 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 e1dfc52a..374da4cd 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 @@ -969,7 +969,7 @@ public class HierarchicalShardSyncerTest { parentShardIds.add(shard.adjacentParentShardId()); } return new Lease(shard.shardId(), leaseOwner, 0L, UUID.randomUUID(), 0L, checkpoint, null, 0L, - parentShardIds); + parentShardIds, null); }).collect(Collectors.toList()); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java index ee38116f..591e7db0 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java @@ -34,9 +34,10 @@ public class LeaseBuilder { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); + private byte[] pendingCheckpointState; public Lease build() { return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds); + checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, pendingCheckpointState); } } \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java index 61cba722..f22e6e4d 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 @@ -55,7 +55,7 @@ public class DynamoDBLeaseRenewerTest { private LeaseRefresher leaseRefresher; private static Lease newLease(String leaseKey) { - return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>()); + return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>(), null); } @Before From a61890ab40358d78d4994c0d14c9a0c509af2a67 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 06:16:04 -0400 Subject: [PATCH 056/159] Adding integration test for serializing application info --- ...namoDBLeaseCoordinatorIntegrationTest.java | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java index 3af33c69..d89c010e 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinatorIntegrationTest.java @@ -127,16 +127,37 @@ public class DynamoDBLeaseCoordinatorIntegrationTest { } assertNotNull(lease); - ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber("newCheckpoint"); + final ExtendedSequenceNumber initialCheckpoint = new ExtendedSequenceNumber("initialCheckpoint"); + final ExtendedSequenceNumber pendingCheckpoint = new ExtendedSequenceNumber("pendingCheckpoint"); + final ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber("newCheckpoint"); + final byte[] checkpointState = "checkpointState".getBytes(); + // lease's leaseCounter is wrong at this point, but it shouldn't matter. + assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), initialCheckpoint, lease.concurrencyToken())); + + final Lease leaseFromDDBAtInitialCheckpoint = leaseRefresher.getLease(lease.leaseKey()); + lease.leaseCounter(lease.leaseCounter() + 1); + lease.checkpoint(initialCheckpoint); + lease.leaseOwner(coordinator.workerIdentifier()); + assertEquals(lease, leaseFromDDBAtInitialCheckpoint); + + dynamoDBCheckpointer.prepareCheckpoint(lease.leaseKey(), pendingCheckpoint, lease.concurrencyToken().toString(), checkpointState); + + final Lease leaseFromDDBAtPendingCheckpoint = leaseRefresher.getLease(lease.leaseKey()); + lease.leaseCounter(lease.leaseCounter() + 1); + lease.checkpoint(initialCheckpoint); + lease.pendingCheckpoint(pendingCheckpoint); + lease.pendingCheckpointState(checkpointState); + assertEquals(lease, leaseFromDDBAtPendingCheckpoint); + assertTrue(dynamoDBCheckpointer.setCheckpoint(lease.leaseKey(), newCheckpoint, lease.concurrencyToken())); - Lease fromDynamo = leaseRefresher.getLease(lease.leaseKey()); - + final Lease leaseFromDDBAtNewCheckpoint = leaseRefresher.getLease(lease.leaseKey()); lease.leaseCounter(lease.leaseCounter() + 1); lease.checkpoint(newCheckpoint); - lease.leaseOwner(coordinator.workerIdentifier()); - assertEquals(lease, fromDynamo); + lease.pendingCheckpoint(null); + lease.pendingCheckpointState(null); + assertEquals(lease, leaseFromDDBAtNewCheckpoint); } /** From 80df8ce4390853a2ea24f43928e854ecb3403995 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 7 Apr 2020 06:22:22 -0400 Subject: [PATCH 057/159] Cleaning up unused imports --- .../java/software/amazon/kinesis/checkpoint/Checkpoint.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java index 7b7bea9e..91f01d0b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java @@ -18,8 +18,6 @@ import lombok.Data; import lombok.experimental.Accessors; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.Arrays; - /** * A class encapsulating the 2 pieces of state stored in a checkpoint. */ From 64469f419988740c6a30a54a1d5e56fddf25094a Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 10 Apr 2020 05:55:45 -0400 Subject: [PATCH 058/159] PR feedback --- .../amazon/kinesis/checkpoint/Checkpoint.java | 5 + .../ShardRecordProcessorCheckpointer.java | 9 ++ .../software/amazon/kinesis/leases/Lease.java | 13 ++- .../kinesis/lifecycle/InitializeTask.java | 1 + .../lifecycle/events/InitializationInput.java | 8 ++ .../kinesis/processor/Checkpointer.java | 13 +++ .../RecordProcessorCheckpointer.java | 105 +++++++++++++++++- 7 files changed, 152 insertions(+), 2 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java index 91f01d0b..f5af81e3 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/Checkpoint.java @@ -28,6 +28,11 @@ public class Checkpoint { private final ExtendedSequenceNumber pendingCheckpoint; private final byte[] pendingCheckpointState; + @Deprecated + public Checkpoint(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) { + this(checkpoint, pendingCheckpoint, null); + } + /** * Constructor. * 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 ec9ff11d..4e831c4b 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 @@ -140,6 +140,9 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi this.largestPermittedCheckpointValue.subSequenceNumber()); } + /** + * {@inheritDoc} + */ @Override public PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { return prepareCheckpoint(largestPermittedCheckpointValue.sequenceNumber(), applicationState); @@ -177,6 +180,9 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi return prepareCheckpoint(sequenceNumber, 0); } + /** + * {@inheritDoc} + */ @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { @@ -192,6 +198,9 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi return prepareCheckpoint(sequenceNumber, subSequenceNumber, null); } + /** + * {@inheritDoc} + */ @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { 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 a7ed666c..15f79739 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 @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; @NoArgsConstructor @Getter @Accessors(fluent = true) -@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos"}) +@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "pendingCheckpointState"}) @ToString public class Lease { /* @@ -84,6 +84,8 @@ public class Lease { private ExtendedSequenceNumber pendingCheckpoint; /** + * Last pending application state. Deliberately excluded from hashCode and equals. + * * @return pending checkpoint state, possibly null. */ private byte[] pendingCheckpointState; @@ -105,6 +107,15 @@ public class Lease { lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.pendingCheckpointState()); } + @Deprecated + public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, + final UUID concurrencyToken, final Long lastCounterIncrementNanos, + final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, + final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds) { + this(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, pendingCheckpoint, + ownerSwitchesSinceCheckpoint, parentShardIds, null); + } + public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, final UUID concurrencyToken, final Long lastCounterIncrementNanos, final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, 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 e11eebfa..4108dd9b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/InitializeTask.java @@ -92,6 +92,7 @@ public class InitializeTask implements ConsumerTask { .shardId(shardInfo.shardId()) .extendedSequenceNumber(initialCheckpoint) .pendingCheckpointSequenceNumber(initialCheckpointObject.pendingCheckpoint()) + .pendingCheckpointState(initialCheckpointObject.pendingCheckpointState()) .build(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java index d6c586aa..3717a805 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/InitializationInput.java @@ -47,4 +47,12 @@ public class InitializationInput { * completing the checkpoint. */ private final ExtendedSequenceNumber pendingCheckpointSequenceNumber; + + /** + * The last pending application state of the previous record processor. May be null. + * + * This will only be set if the previous record processor had prepared a checkpoint, but lost its lease before + * completing the checkpoint. + */ + private final byte[] pendingCheckpointState; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java index 7f3c947e..2ffadc06 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/Checkpointer.java @@ -73,6 +73,19 @@ public interface Checkpointer { void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) throws KinesisClientLibException; + /** + * Record intent to checkpoint for a shard. Upon failover, the pendingCheckpoint and pendingCheckpointState will be + * passed to the new ShardRecordProcessor's initialize() method. + * + * @param leaseKey Checkpoint is specified for this shard. + * @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number) + * @param concurrencyToken Used with conditional writes to prevent stale updates + * (e.g. if there was a fail over to a different record processor, we don't want to + * overwrite it's checkpoint) + * @param pendingCheckpointState Serialized application state at the pending checkpoint. + * + * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint + */ void prepareCheckpoint(String leaseKey, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken, byte[] pendingCheckpointState) throws KinesisClientLibException; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java index d42cbb9e..34b2930c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/RecordProcessorCheckpointer.java @@ -144,6 +144,29 @@ public interface RecordProcessorCheckpointer { PreparedCheckpointer prepareCheckpoint() throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + /** + * This method will record a pending checkpoint at the last data record that was delivered to the record processor. + * If the application fails over between calling prepareCheckpoint() and checkpoint(), the init() method of the next + * IRecordProcessor for this shard will be informed of the prepared sequence number and application state. + * + * Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having + * side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete. + * Use the sequence number and application state passed in to init() to behave idempotently. + * + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + */ PreparedCheckpointer prepareCheckpoint(byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; @@ -176,7 +199,35 @@ public interface RecordProcessorCheckpointer { PreparedCheckpointer prepareCheckpoint(Record record) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; - + /** + * This method will record a pending checkpoint at the at the provided record. This method is analogous to + * {@link #prepareCheckpoint()} but provides the ability to specify the record and application state at which to + * prepare the checkpoint. + * + * @param record A record at which to prepare checkpoint in this shard. + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * + * Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having + * side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete. + * Use the sequence number and application state passed in to init() to behave idempotently. + * + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; @@ -206,6 +257,31 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + /** + * This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to + * {@link #prepareCheckpoint()} but provides the ability to specify the sequence number and application state + * at which to checkpoint. + * + * @param sequenceNumber A sequence number at which to prepare checkpoint in this shard. + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; @@ -238,6 +314,33 @@ public interface RecordProcessorCheckpointer { throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + /** + * This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for + * aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()} + * but provides the ability to specify the sequence number, subsequence number, and application state at which to + * checkpoint. + * + * @param sequenceNumber A sequence number at which to prepare checkpoint in this shard. + * @param subSequenceNumber A subsequence number at which to prepare checkpoint within this shard. + * @param applicationState arbitrary application state that will be passed to the next record processor that + * processes the shard. + * + * @return an PreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this ShardRecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ PreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; From d31758727c3d3a1c56271b1790ca62ca6093f62c Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 14 Apr 2020 04:37:11 -0400 Subject: [PATCH 059/159] Fixing bug --- .../kinesis/checkpoint/ShardRecordProcessorCheckpointer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 4e831c4b..8ecf58ec 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 @@ -186,7 +186,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi @Override public PreparedCheckpointer prepareCheckpoint(String sequenceNumber, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { - return prepareCheckpoint(sequenceNumber, 0, null); + return prepareCheckpoint(sequenceNumber, 0, applicationState); } /** From 4782e43c3a24c45d78b1a69b165c095f8b90dcc9 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 15 Apr 2020 19:26:53 -0400 Subject: [PATCH 060/159] PR comments --- .../kinesis/checkpoint/ShardRecordProcessorCheckpointer.java | 3 +-- 1 file changed, 1 insertion(+), 2 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 8ecf58ec..31506304 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 @@ -338,8 +338,7 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi } try { - checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken(), applicationState - ); + checkpointer.prepareCheckpoint(ShardInfo.getLeaseKey(shardInfo), newPrepareCheckpoint, shardInfo.concurrencyToken(), applicationState); } catch (ThrottlingException | ShutdownException | InvalidStateException | KinesisClientLibDependencyException e) { throw e; From 22fa12abfaa4b38ec5662083fb298e0a2d36f06f Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 15 Apr 2020 19:29:17 -0400 Subject: [PATCH 061/159] Java docs --- .../kinesis/checkpoint/ShardRecordProcessorCheckpointer.java | 3 +++ 1 file changed, 3 insertions(+) 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 31506304..fd375264 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/checkpoint/ShardRecordProcessorCheckpointer.java @@ -148,6 +148,9 @@ public class ShardRecordProcessorCheckpointer implements RecordProcessorCheckpoi return prepareCheckpoint(largestPermittedCheckpointValue.sequenceNumber(), applicationState); } + /** + * {@inheritDoc} + */ @Override public PreparedCheckpointer prepareCheckpoint(Record record, byte[] applicationState) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { // From a97a617f359914379b2b5595662dd82ac3578a54 Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Fri, 17 Apr 2020 13:20:41 -0700 Subject: [PATCH 062/159] Add DataFetcher and ShardDetector interface changes (#9) --- .../amazon/kinesis/coordinator/Scheduler.java | 40 +++--- .../kinesis/leases/KinesisShardDetector.java | 4 +- .../kinesis/leases/LeaseManagementConfig.java | 115 ++++++++-------- .../amazon/kinesis/leases/ShardDetector.java | 4 +- .../DynamoDBLeaseManagementFactory.java | 22 ++-- .../kinesis/lifecycle/ShutdownTask.java | 10 +- .../lifecycle/events/ProcessRecordsInput.java | 1 + .../retrieval/DataFetcherProviderConfig.java | 48 +++++++ .../GetRecordsRetrievalStrategy.java | 28 +++- .../KinesisDataFetcherProviderConfig.java | 45 +++++++ .../kinesis/retrieval/RetrievalConfig.java | 27 +++- .../retrieval/RetrievalSpecificConfig.java | 5 +- .../retrieval/polling/DataFetcher.java | 124 ++++++++++++++++++ .../retrieval/polling/KinesisDataFetcher.java | 109 +++++++++------ .../retrieval/polling/PollingConfig.java | 35 ++++- .../polling/PrefetchRecordsPublisher.java | 21 ++- .../SynchronousBlockingRetrievalFactory.java | 59 +++++++-- ...ynchronousGetRecordsRetrievalStrategy.java | 11 +- ...ynchronousPrefetchingRetrievalFactory.java | 12 +- .../polling/PrefetchRecordsPublisherTest.java | 4 +- .../polling/RecordsFetcherFactoryTest.java | 7 +- 21 files changed, 545 insertions(+), 186 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/DataFetcher.java 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 cb4b4579..4d6c1fd7 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -15,6 +15,10 @@ package software.amazon.kinesis.coordinator; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; + +import io.reactivex.plugins.RxJavaPlugins; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -29,14 +33,12 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Stopwatch; -import io.reactivex.plugins.RxJavaPlugins; import lombok.AccessLevel; import lombok.Getter; import lombok.NoArgsConstructor; @@ -50,6 +52,7 @@ import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -59,9 +62,7 @@ import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardPrioritization; -import software.amazon.kinesis.leases.ShardSyncTask; import software.amazon.kinesis.leases.ShardSyncTaskManager; -import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseCoordinator; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseSerializer; import software.amazon.kinesis.leases.dynamodb.DynamoDBMultiStreamLeaseSerializer; @@ -77,7 +78,6 @@ import software.amazon.kinesis.lifecycle.ShutdownNotification; import software.amazon.kinesis.lifecycle.ShutdownReason; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.CloudWatchMetricsFactory; -import software.amazon.kinesis.metrics.MetricsCollectingTaskDecorator; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.processor.Checkpointer; @@ -89,9 +89,6 @@ import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadLocalRandom; - /** * */ @@ -326,6 +323,7 @@ public class Scheduler implements Runnable { if (shouldInitiateLeaseSync()) { log.info("Worker {} is initiating the lease sync.", leaseManagementConfig.workerIdentifier()); leaderElectedPeriodicShardSyncManager.syncShardsOnce(); + } } else { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); @@ -551,7 +549,6 @@ public class Scheduler implements Runnable { * Requests a graceful shutdown of the worker, notifying record processors, that implement * {@link ShutdownNotificationAware}, of the impending shutdown. This gives the record processor a final chance to * checkpoint. - * * This will only create a single shutdown future. Additional attempts to start a graceful shutdown will return the * previous future. * @@ -578,8 +575,8 @@ public class Scheduler implements Runnable { * * * @return a future that will be set once the shutdown has completed. True indicates that the graceful shutdown - * completed successfully. A false value indicates that a non-exception case caused the shutdown process to - * terminate early. + * completed successfully. A false value indicates that a non-exception case caused the shutdown process to + * terminate early. */ public Future startGracefulShutdown() { synchronized (this) { @@ -596,9 +593,8 @@ public class Scheduler implements Runnable { * shutdowns in your own executor, or execute the shutdown synchronously. * * @return a callable that run the graceful shutdown process. This may return a callable that return true if the - * graceful shutdown has already been completed. - * @throws IllegalStateException - * thrown by the callable if another callable has already started the shutdown process. + * graceful shutdown has already been completed. + * @throws IllegalStateException thrown by the callable if another callable has already started the shutdown process. */ public Callable createGracefulShutdownCallable() { if (shutdownComplete()) { @@ -740,12 +736,11 @@ public class Scheduler implements Runnable { /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. * - * @param shardInfo - * Kinesis shard info + * @param shardInfo Kinesis shard info * @return ShardConsumer for the shard */ ShardConsumer createOrGetShardConsumer(@NonNull final ShardInfo shardInfo, - @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { + @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { ShardConsumer consumer = shardInfoShardConsumerMap.get(shardInfo); // Instantiate a new consumer if we don't have one, or the one we // had was from an earlier @@ -766,10 +761,10 @@ public class Scheduler implements Runnable { } protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, - @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { + @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory); ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, - checkpoint); + checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt()); @@ -806,7 +801,6 @@ public class Scheduler implements Runnable { /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. - *

* This method relies on ShardInfo.equals() method returning true for ShardInfo objects which may have been * instantiated with parentShardIds in a different order (and rest of the fields being the equal). For example * shardInfo1.equals(shardInfo2) should return true with shardInfo1 and shardInfo2 defined as follows. ShardInfo @@ -851,7 +845,7 @@ public class Scheduler implements Runnable { private StreamIdentifier getStreamIdentifier(Optional streamIdentifierString) { final StreamIdentifier streamIdentifier; - if(streamIdentifierString.isPresent()) { + if (streamIdentifierString.isPresent()) { streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierString.get()); } else { Validate.isTrue(!isMultiStreamMode, "Should not be in MultiStream Mode"); 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 c0c3bdee..a29c5ce4 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 @@ -26,15 +26,13 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; - -import org.apache.commons.lang3.StringUtils; - import lombok.AccessLevel; import lombok.Getter; import lombok.NonNull; import lombok.Synchronized; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.services.kinesis.model.KinesisException; import software.amazon.awssdk.services.kinesis.model.LimitExceededException; 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 2a5a0b1e..acaa8de0 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java @@ -15,17 +15,15 @@ package software.amazon.kinesis.leases; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + import java.time.Duration; -import java.util.HashMap; -import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; - -import com.google.common.util.concurrent.ThreadFactoryBuilder; - +import java.util.function.Function; import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; @@ -35,11 +33,11 @@ import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseManagementFactory; import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.NullMetricsFactory; -import software.amazon.kinesis.processor.MultiStreamTracker; /** * Used by the KCL to configure lease management. @@ -145,6 +143,11 @@ public class LeaseManagementConfig { */ private int initialLeaseTableWriteCapacity = 10; + /** + * Configurable functional interface to override the existing shardDetector. + */ + private Function customShardDetectorProvider; + /** * The size of the thread pool to create for the lease renewer to use. * @@ -291,30 +294,30 @@ public class LeaseManagementConfig { if (leaseManagementFactory == null) { Validate.notEmpty(streamName(), "Stream name is empty"); leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), - streamName(), - dynamoDBClient(), - tableName(), - workerIdentifier(), - executorService(), - initialPositionInStream(), - failoverTimeMillis(), - epsilonMillis(), - maxLeasesForWorker(), - maxLeasesToStealAtOneTime(), - maxLeaseRenewalThreads(), - cleanupLeasesUponShardCompletion(), - ignoreUnexpectedChildShards(), - shardSyncIntervalMillis(), - consistentReads(), - listShardsBackoffTimeInMillis(), - maxListShardsRetryAttempts(), - maxCacheMissesBeforeReload(), - listShardsCacheAllowedAgeInSeconds(), - cacheMissWarningModulus(), - initialLeaseTableReadCapacity(), - initialLeaseTableWriteCapacity(), - hierarchicalShardSyncer(), - tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode()); + streamName(), + dynamoDBClient(), + tableName(), + workerIdentifier(), + executorService(), + initialPositionInStream(), + failoverTimeMillis(), + epsilonMillis(), + maxLeasesForWorker(), + maxLeasesToStealAtOneTime(), + maxLeaseRenewalThreads(), + cleanupLeasesUponShardCompletion(), + ignoreUnexpectedChildShards(), + shardSyncIntervalMillis(), + consistentReads(), + listShardsBackoffTimeInMillis(), + maxListShardsRetryAttempts(), + maxCacheMissesBeforeReload(), + listShardsCacheAllowedAgeInSeconds(), + cacheMissWarningModulus(), + initialLeaseTableReadCapacity(), + initialLeaseTableWriteCapacity(), + hierarchicalShardSyncer(), + tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode()); } return leaseManagementFactory; } @@ -328,31 +331,32 @@ public class LeaseManagementConfig { public LeaseManagementFactory leaseManagementFactory(final LeaseSerializer leaseSerializer, boolean isMultiStreamingMode) { if(leaseManagementFactory == null) { leaseManagementFactory = new DynamoDBLeaseManagementFactory(kinesisClient(), - dynamoDBClient(), - tableName(), - workerIdentifier(), - executorService(), - failoverTimeMillis(), - epsilonMillis(), - maxLeasesForWorker(), - maxLeasesToStealAtOneTime(), - maxLeaseRenewalThreads(), - cleanupLeasesUponShardCompletion(), - ignoreUnexpectedChildShards(), - shardSyncIntervalMillis(), - consistentReads(), - listShardsBackoffTimeInMillis(), - maxListShardsRetryAttempts(), - maxCacheMissesBeforeReload(), - listShardsCacheAllowedAgeInSeconds(), - cacheMissWarningModulus(), - initialLeaseTableReadCapacity(), - initialLeaseTableWriteCapacity(), - hierarchicalShardSyncer(isMultiStreamingMode), - tableCreatorCallback(), - dynamoDbRequestTimeout(), - billingMode(), - leaseSerializer); + dynamoDBClient(), + tableName(), + workerIdentifier(), + executorService(), + failoverTimeMillis(), + epsilonMillis(), + maxLeasesForWorker(), + maxLeasesToStealAtOneTime(), + maxLeaseRenewalThreads(), + cleanupLeasesUponShardCompletion(), + ignoreUnexpectedChildShards(), + shardSyncIntervalMillis(), + consistentReads(), + listShardsBackoffTimeInMillis(), + maxListShardsRetryAttempts(), + maxCacheMissesBeforeReload(), + listShardsCacheAllowedAgeInSeconds(), + cacheMissWarningModulus(), + initialLeaseTableReadCapacity(), + initialLeaseTableWriteCapacity(), + hierarchicalShardSyncer(isMultiStreamingMode), + tableCreatorCallback(), + dynamoDbRequestTimeout(), + billingMode(), + leaseSerializer, + customShardDetectorProvider()); } return leaseManagementFactory; } @@ -366,5 +370,4 @@ public class LeaseManagementConfig { this.leaseManagementFactory = leaseManagementFactory; return this; } - } 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 1b2822ee..2967a9fb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java @@ -15,12 +15,11 @@ package software.amazon.kinesis.leases; +import java.util.List; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.kinesis.common.StreamIdentifier; -import java.util.List; - /** * */ @@ -34,5 +33,4 @@ public interface ShardDetector { default StreamIdentifier streamIdentifier() { throw new UnsupportedOperationException("StreamName not available"); } - } 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 8c09af52..44879c1c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -17,7 +17,7 @@ package software.amazon.kinesis.leases.dynamodb; import java.time.Duration; import java.util.concurrent.ExecutorService; - +import java.util.function.Function; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; @@ -61,6 +61,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @NonNull private StreamConfig streamConfig; + private Function customShardDetectorProvider; + private final long failoverTimeMillis; private final long epsilonMillis; private final int maxLeasesForWorker; @@ -231,7 +233,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { /** * Constructor. - * + * * @param kinesisClient * @param streamName * @param dynamoDBClient @@ -365,7 +367,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param dynamoDbRequestTimeout * @param billingMode */ - public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final StreamConfig streamConfig, + 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, @@ -382,7 +384,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer); + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer, + null); this.streamConfig = streamConfig; } @@ -425,7 +428,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, - Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { + Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer, + Function customShardDetectorProvider) { this.kinesisClient = kinesisClient; this.dynamoDBClient = dynamoDBClient; this.tableName = tableName; @@ -452,6 +456,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.dynamoDbRequestTimeout = dynamoDbRequestTimeout; this.billingMode = billingMode; this.leaseSerializer = leaseSerializer; + this.customShardDetectorProvider = customShardDetectorProvider; } @Override @@ -522,8 +527,9 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { */ @Override public ShardDetector createShardDetector(StreamConfig streamConfig) { - return new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis, - maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, - cacheMissWarningModulus, dynamoDbRequestTimeout); + return customShardDetectorProvider != null ? customShardDetectorProvider.apply(streamConfig) : + new KinesisShardDetector(kinesisClient, streamConfig.streamIdentifier(), listShardsBackoffTimeMillis, + maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, + cacheMissWarningModulus, dynamoDbRequestTimeout); } } 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 33eb4497..aab984eb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -16,6 +16,8 @@ package software.amazon.kinesis.lifecycle; import com.google.common.annotations.VisibleForTesting; +import java.util.List; +import java.util.function.Function; import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -26,11 +28,11 @@ import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; -import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; @@ -38,16 +40,12 @@ import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; import software.amazon.kinesis.lifecycle.events.ShardEndedInput; import software.amazon.kinesis.metrics.MetricsFactory; -import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; - -import java.util.List; -import java.util.function.Function; - /** * Task for invoking the ShardRecordProcessor shutdown() callback. */ diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java index 3bfcd514..1ce9239b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/events/ProcessRecordsInput.java @@ -57,6 +57,7 @@ public class ProcessRecordsInput { * The records received from Kinesis. These records may have been de-aggregated if they were published by the KPL. */ private List records; + /** * A checkpointer that the {@link ShardRecordProcessor} can use to checkpoint its progress. */ diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java new file mode 100644 index 00000000..b5c7b23e --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataFetcherProviderConfig.java @@ -0,0 +1,48 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.retrieval; + +import java.time.Duration; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.metrics.MetricsFactory; + +public interface DataFetcherProviderConfig { + + /** + * Gets stream identifier for dataFetcher. + */ + StreamIdentifier getStreamIdentifier(); + + /** + * Gets shard id. + */ + String getShardId(); + + /** + * Gets current instance of metrics factory. + */ + MetricsFactory getMetricsFactory(); + + /** + * Gets current max records allowed to process at a given time. + */ + Integer getMaxRecords(); + + /** + * Gets timeout for kinesis request. + */ + Duration getKinesisRequestTimeout(); +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java index ca0487f3..3ff8e620 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/GetRecordsRetrievalStrategy.java @@ -14,7 +14,9 @@ */ package software.amazon.kinesis.retrieval; +import java.util.Optional; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.kinesis.retrieval.polling.DataFetcher; import software.amazon.kinesis.retrieval.polling.KinesisDataFetcher; /** @@ -41,15 +43,33 @@ public interface GetRecordsRetrievalStrategy { /** * Returns whether this strategy has been shutdown. - * + * * @return true if the strategy has been shutdown, false otherwise. */ boolean isShutdown(); /** - * Returns the KinesisDataFetcher used to records from Kinesis. - * - * @return KinesisDataFetcher + * Returns a DataFetcher used to records from Kinesis. + * + * @return DataFetcher */ KinesisDataFetcher getDataFetcher(); + + /** + * Returns a DataFetcher override if applicable, else empty for retrieving records from Kinesis. + * + * @return Optional + */ + default Optional getDataFetcherOverride() { + return Optional.empty(); + } + + /** + * Returns a dataFetcher by first checking for an override if it exists, else using the default data fetcher. + * + * @return DataFetcher + */ + default DataFetcher dataFetcher() { + return getDataFetcherOverride().orElse(getDataFetcher()); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java new file mode 100644 index 00000000..7cf6cdcf --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/KinesisDataFetcherProviderConfig.java @@ -0,0 +1,45 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.retrieval; + +import java.time.Duration; +import lombok.Data; +import lombok.NonNull; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.metrics.MetricsFactory; + + +/** + * Configuration needed for custom data fetchers + */ +@Data +public class KinesisDataFetcherProviderConfig implements DataFetcherProviderConfig { + + @NonNull + private StreamIdentifier streamIdentifier; + + @NonNull + private String shardId; + + @NonNull + private MetricsFactory metricsFactory; + + @NonNull + private Integer maxRecords; + + @NonNull + private Duration kinesisRequestTimeout; +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalConfig.java index 746fdc19..5f22411a 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 @@ -29,11 +29,15 @@ import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.retrieval.fanout.FanOutConfig; +import software.amazon.kinesis.retrieval.polling.PollingConfig; /** * Used by the KCL to configure the retrieval of records from Kinesis. */ -@Getter @Setter @ToString @EqualsAndHashCode +@Getter +@Setter +@ToString +@EqualsAndHashCode @Accessors(fluent = true) public class RetrievalConfig { /** @@ -52,6 +56,7 @@ public class RetrievalConfig { @NonNull private final String applicationName; + /** * AppStreamTracker either for multi stream tracking or single stream */ @@ -91,7 +96,7 @@ public class RetrievalConfig { private RetrievalFactory retrievalFactory; public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull String streamName, - @NonNull String applicationName) { + @NonNull String applicationName) { this.kinesisClient = kinesisAsyncClient; this.appStreamTracker = Either .right(new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStreamExtended)); @@ -99,7 +104,7 @@ public class RetrievalConfig { } public RetrievalConfig(@NonNull KinesisAsyncClient kinesisAsyncClient, @NonNull MultiStreamTracker multiStreamTracker, - @NonNull String applicationName) { + @NonNull String applicationName) { this.kinesisClient = kinesisAsyncClient; this.appStreamTracker = Either.left(multiStreamTracker); this.applicationName = applicationName; @@ -117,17 +122,29 @@ public class RetrievalConfig { } public RetrievalFactory retrievalFactory() { - if (retrievalFactory == null) { if (retrievalSpecificConfig == null) { retrievalSpecificConfig = new FanOutConfig(kinesisClient()) .applicationName(applicationName()); retrievalSpecificConfig = appStreamTracker.map(multiStreamTracker -> retrievalSpecificConfig, - streamConfig -> ((FanOutConfig)retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName())); + streamConfig -> ((FanOutConfig) retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName())); } + retrievalFactory = retrievalSpecificConfig.retrievalFactory(); } + validateConfig(); return retrievalFactory; } + private void validateConfig() { + 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: multistream enabled with streamName or single stream with no streamName"); + } + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java index 5ab982bf..30562994 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalSpecificConfig.java @@ -15,10 +15,13 @@ package software.amazon.kinesis.retrieval; +import java.util.function.Function; +import software.amazon.kinesis.retrieval.polling.DataFetcher; + public interface RetrievalSpecificConfig { /** * Creates and returns a retrieval factory for the specific configuration - * + * * @return a retrieval factory that can create an appropriate retriever */ RetrievalFactory retrievalFactory(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/DataFetcher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/DataFetcher.java new file mode 100644 index 00000000..ae1c6f30 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/DataFetcher.java @@ -0,0 +1,124 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.retrieval.polling; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import lombok.NonNull; +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.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.retrieval.DataFetcherResult; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +public interface DataFetcher { + /** + * Get records from the current position in the stream (up to maxRecords). + * + * @return list of records of up to maxRecords size + */ + DataFetcherResult getRecords(); + + /** + * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number. + * + * @param initialCheckpoint Current checkpoint sequence number for this shard. + * @param initialPositionInStream The initialPositionInStream. + */ + void initialize(String initialCheckpoint, + InitialPositionInStreamExtended initialPositionInStream); + + /** + * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number as an + * ExtendedSequenceNumber. + * + * @param initialCheckpoint Current checkpoint sequence number for this shard. + * @param initialPositionInStream The initialPositionInStream. + */ + void initialize(ExtendedSequenceNumber initialCheckpoint, + InitialPositionInStreamExtended initialPositionInStream); + + /** + * Advances this KinesisDataFetcher's internal iterator to be at the passed-in sequence number. + * + * @param sequenceNumber advance the iterator to the record at this sequence number. + * @param initialPositionInStream The initialPositionInStream. + */ + void advanceIteratorTo(String sequenceNumber, + InitialPositionInStreamExtended initialPositionInStream); + + /** + * Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last + * records call. + */ + void restartIterator(); + + /** + * Resets the iterator by setting shardIterator, sequenceNumber and the position in the stream. + * + * @param shardIterator set the current shard iterator. + * @param sequenceNumber reset the iterator to the record at this sequence number. + * @param initialPositionInStream the current position in the stream to reset the iterator to. + */ + void resetIterator(String shardIterator, String sequenceNumber, InitialPositionInStreamExtended initialPositionInStream); + + /** + * Retrieves the response based on the request. + * + * @param request the current get records request used to receive a response. + * @return GetRecordsResponse response for getRecords + */ + GetRecordsResponse getGetRecordsResponse(GetRecordsRequest request) throws Exception; + + /** + * Retrieves the next get records request based on the current iterator. + * + * @param nextIterator specify the iterator to get the next record request + * @return {@link GetRecordsRequest} + */ + GetRecordsRequest getGetRecordsRequest(String nextIterator); + + /** + * Gets the next iterator based on the request. + * + * @param request used to obtain the next shard iterator + * @return next iterator string + */ + String getNextIterator(GetShardIteratorRequest request) throws ExecutionException, InterruptedException, TimeoutException; + + /** + * Gets the next set of records based on the iterator. + * + * @param nextIterator specified shard iterator for getting the next set of records + * @return {@link GetRecordsResponse} + */ + GetRecordsResponse getRecords(@NonNull String nextIterator); + + /** + * Get the current account and stream information. + * + * @return {@link StreamIdentifier} + */ + StreamIdentifier getStreamIdentifier(); + + /** + * Checks if shardEnd is reached. + * @return boolean to determine whether shard end is reached + */ + boolean isShardEndReached(); +} 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 a96e2134..82b31915 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 @@ -14,20 +14,18 @@ */ package software.amazon.kinesis.retrieval.polling; +import com.google.common.collect.Iterables; + import java.time.Duration; import java.util.Collections; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; - -import org.apache.commons.lang3.StringUtils; - -import com.google.common.collect.Iterables; - import lombok.AccessLevel; import lombok.Data; import lombok.Getter; import lombok.NonNull; 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.GetRecordsRequest; @@ -47,8 +45,10 @@ import software.amazon.kinesis.metrics.MetricsLevel; import software.amazon.kinesis.metrics.MetricsScope; 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.IteratorBuilder; +import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RetryableRetrievalException; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @@ -57,7 +57,7 @@ import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; */ @Slf4j @KinesisClientInternalApi -public class KinesisDataFetcher { +public class KinesisDataFetcher implements DataFetcher { private static final String METRICS_PREFIX = "KinesisDataFetcher"; private static final String OPERATION = "ProcessTask"; @@ -76,33 +76,39 @@ public class KinesisDataFetcher { @Deprecated public KinesisDataFetcher(KinesisAsyncClient kinesisClient, String streamName, String shardId, int maxRecords, MetricsFactory metricsFactory) { - this(kinesisClient, StreamIdentifier.singleStreamInstance(streamName), shardId, maxRecords, metricsFactory, PollingConfig.DEFAULT_REQUEST_TIMEOUT); + this(kinesisClient, new KinesisDataFetcherProviderConfig( + StreamIdentifier.singleStreamInstance(streamName), + shardId, + metricsFactory, + maxRecords, + PollingConfig.DEFAULT_REQUEST_TIMEOUT + )); } /** - * Constructs KinesisDataFetcher. - * @param kinesisClient - * @param streamIdentifier - * @param shardId - * @param maxRecords - * @param metricsFactory - * @param maxFutureWait - */ - public KinesisDataFetcher(KinesisAsyncClient kinesisClient, StreamIdentifier streamIdentifier, String shardId, int maxRecords, MetricsFactory metricsFactory, Duration maxFutureWait) { - this.kinesisClient = kinesisClient; - this.streamIdentifier = streamIdentifier; - this.shardId = shardId; - this.maxRecords = maxRecords; - this.metricsFactory = metricsFactory; - this.maxFutureWait = maxFutureWait; - this.streamAndShardId = streamIdentifier.serialize() + ":" + shardId; - } - - /** Note: This method has package level access for testing purposes. + * Note: This method has package level access for testing purposes. + * * @return nextIterator */ @Getter(AccessLevel.PACKAGE) private String nextIterator; + + /** + * Constructs KinesisDataFetcher. + * + * @param kinesisClient + * @param kinesisDataFetcherProviderConfig + */ + public KinesisDataFetcher(KinesisAsyncClient kinesisClient, DataFetcherProviderConfig kinesisDataFetcherProviderConfig) { + this.kinesisClient = kinesisClient; + this.maxFutureWait = kinesisDataFetcherProviderConfig.getKinesisRequestTimeout(); + this.maxRecords = kinesisDataFetcherProviderConfig.getMaxRecords(); + this.metricsFactory = kinesisDataFetcherProviderConfig.getMetricsFactory(); + this.shardId = kinesisDataFetcherProviderConfig.getShardId(); + this.streamIdentifier = kinesisDataFetcherProviderConfig.getStreamIdentifier(); + this.streamAndShardId = streamIdentifier.serialize() + ":" + shardId; + } + @Getter private boolean isShardEndReached; private boolean isInitialized; @@ -114,6 +120,7 @@ public class KinesisDataFetcher { * * @return list of records of up to maxRecords size */ + @Override public DataFetcherResult getRecords() { if (!isInitialized) { throw new IllegalArgumentException("KinesisDataFetcher.records called before initialization."); @@ -187,6 +194,7 @@ public class KinesisDataFetcher { * @param initialCheckpoint Current checkpoint sequence number for this shard. * @param initialPositionInStream The initialPositionInStream. */ + @Override public void initialize(final String initialCheckpoint, final InitialPositionInStreamExtended initialPositionInStream) { log.info("Initializing shard {} with {}", streamAndShardId, initialCheckpoint); @@ -194,6 +202,7 @@ public class KinesisDataFetcher { isInitialized = true; } + @Override public void initialize(final ExtendedSequenceNumber initialCheckpoint, final InitialPositionInStreamExtended initialPositionInStream) { log.info("Initializing shard {} with {}", streamAndShardId, initialCheckpoint.sequenceNumber()); @@ -207,6 +216,7 @@ public class KinesisDataFetcher { * @param sequenceNumber advance the iterator to the record at this sequence number. * @param initialPositionInStream The initialPositionInStream. */ + @Override public void advanceIteratorTo(final String sequenceNumber, final InitialPositionInStreamExtended initialPositionInStream) { if (sequenceNumber == null) { @@ -228,9 +238,7 @@ public class KinesisDataFetcher { try { try { - final GetShardIteratorResponse result = FutureUtils - .resolveOrCancelFuture(kinesisClient.getShardIterator(request), maxFutureWait); - nextIterator = result.shardIterator(); + nextIterator = getNextIterator(request); success = true; } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); @@ -260,6 +268,7 @@ public class KinesisDataFetcher { * Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last * records call. */ + @Override public void restartIterator() { if (StringUtils.isEmpty(lastKnownSequenceNumber) || initialPositionInStream == null) { throw new IllegalStateException( @@ -268,29 +277,49 @@ public class KinesisDataFetcher { advanceIteratorTo(lastKnownSequenceNumber, initialPositionInStream); } + @Override public void resetIterator(String shardIterator, String sequenceNumber, InitialPositionInStreamExtended initialPositionInStream) { this.nextIterator = shardIterator; this.lastKnownSequenceNumber = sequenceNumber; this.initialPositionInStream = initialPositionInStream; } - private GetRecordsResponse getRecords(@NonNull final String nextIterator) { - final AWSExceptionManager exceptionManager = createExceptionManager(); - GetRecordsRequest request = KinesisRequestsBuilder.getRecordsRequestBuilder().shardIterator(nextIterator) + @Override + public GetRecordsResponse getGetRecordsResponse(GetRecordsRequest request) throws ExecutionException, InterruptedException, TimeoutException { + final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request), + maxFutureWait); + if (!isValidResponse(response)) { + throw new RetryableRetrievalException("GetRecords response is not valid for shard: " + streamAndShardId + + ". nextShardIterator: " + response.nextShardIterator() + + ". childShards: " + response.childShards() + ". Will retry GetRecords with the same nextIterator."); + } + return response; + } + + @Override + public GetRecordsRequest getGetRecordsRequest(String nextIterator) { + return KinesisRequestsBuilder.getRecordsRequestBuilder().shardIterator(nextIterator) .limit(maxRecords).build(); + } + + @Override + public String getNextIterator(GetShardIteratorRequest request) throws ExecutionException, InterruptedException, TimeoutException { + final GetShardIteratorResponse result = FutureUtils + .resolveOrCancelFuture(kinesisClient.getShardIterator(request), maxFutureWait); + return result.shardIterator(); + } + + @Override + public GetRecordsResponse getRecords(@NonNull final String nextIterator) { + final AWSExceptionManager exceptionManager = createExceptionManager(); + GetRecordsRequest request = getGetRecordsRequest(nextIterator); final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION); MetricsUtil.addShardId(metricsScope, shardId); - boolean success = false; + boolean success = false ; long startTime = System.currentTimeMillis(); try { - final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request), - maxFutureWait); - if (!isValidResponse(response)) { - throw new RetryableRetrievalException("GetRecords response is not valid for shard: " + streamAndShardId - + ". nextShardIterator: " + response.nextShardIterator() - + ". childShards: " + response.childShards() + ". Will retry GetRecords with the same nextIterator."); - } + final GetRecordsResponse response = getGetRecordsResponse(request); success = true; return response; } catch (ExecutionException e) { 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 cc574506..d8c2405a 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 @@ -17,31 +17,47 @@ 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; +import lombok.Setter; +import lombok.ToString; import lombok.experimental.Accessors; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.kinesis.retrieval.DataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RecordsFetcherFactory; import software.amazon.kinesis.retrieval.RetrievalFactory; import software.amazon.kinesis.retrieval.RetrievalSpecificConfig; @Accessors(fluent = true) -@Data @Getter +@Setter +@ToString +@EqualsAndHashCode public class PollingConfig implements RetrievalSpecificConfig { public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofSeconds(30); + /** + * Configurable functional interface to override the existing DataFetcher. + */ + Function dataFetcherProvider; /** * Name of the Kinesis stream. * * @return String */ - @NonNull - private final String streamName; + private String streamName; + + /** + * @param kinesisClient Client used to access Kinesis services. + */ + public PollingConfig(KinesisAsyncClient kinesisClient) { + this.kinesisClient = kinesisClient; + } /** * Client used to access to Kinesis service. @@ -60,6 +76,15 @@ public class PollingConfig implements RetrievalSpecificConfig { */ private int maxRecords = 10000; + /** + * @param streamName Name of Kinesis stream. + * @param kinesisClient Client used to access Kinesis serivces. + */ + public PollingConfig(String streamName, KinesisAsyncClient kinesisClient) { + this.kinesisClient = kinesisClient; + this.streamName = streamName; + } + /** * The value for how long the ShardConsumer should sleep if no records are returned from the call to * {@link KinesisAsyncClient#getRecords(GetRecordsRequest)}. @@ -105,6 +130,6 @@ public class PollingConfig implements RetrievalSpecificConfig { @Override public RetrievalFactory retrievalFactory() { return new SynchronousBlockingRetrievalFactory(streamName(), kinesisClient(), recordsFetcherFactory, - maxRecords(), kinesisRequestTimeout); + maxRecords(), kinesisRequestTimeout, dataFetcherProvider); } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java index 6e172f08..d9e00669 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -15,6 +15,8 @@ package software.amazon.kinesis.retrieval.polling; +import com.google.common.annotations.VisibleForTesting; + import java.time.Duration; import java.time.Instant; import java.util.List; @@ -25,21 +27,17 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; - import lombok.AccessLevel; +import lombok.Data; import lombok.Getter; +import lombok.NonNull; import lombok.Setter; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.Validate; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; - -import com.google.common.annotations.VisibleForTesting; - -import lombok.Data; -import lombok.NonNull; -import lombok.experimental.Accessors; -import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.awssdk.services.kinesis.model.ExpiredIteratorException; @@ -61,7 +59,6 @@ import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RecordsRetrieved; import software.amazon.kinesis.retrieval.RetryableRetrievalException; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; - import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired; /** @@ -108,7 +105,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { @VisibleForTesting @Getter private final LinkedBlockingQueue prefetchRecordsQueue; private final PrefetchCounters prefetchCounters; - private final KinesisDataFetcher dataFetcher; + private final DataFetcher dataFetcher; private InitialPositionInStreamExtended initialPositionInStreamExtended; private String highestSequenceNumber; @@ -215,7 +212,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { this.maxByteSize = maxByteSize; this.maxRecordsCount = maxRecordsCount; this.publisherSession = new PublisherSession(new LinkedBlockingQueue<>(this.maxPendingProcessRecordsInput), - new PrefetchCounters(), this.getRecordsRetrievalStrategy.getDataFetcher()); + new PrefetchCounters(), this.getRecordsRetrievalStrategy.dataFetcher()); this.executorService = executorService; this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory); this.idleMillisBetweenCalls = idleMillisBetweenCalls; @@ -223,7 +220,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { Validate.notEmpty(operation, "Operation cannot be empty"); this.operation = operation; this.streamAndShardId = - this.getRecordsRetrievalStrategy.getDataFetcher().getStreamIdentifier().serialize() + ":" + shardId; + this.getRecordsRetrievalStrategy.dataFetcher().getStreamIdentifier().serialize() + ":" + shardId; } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java index 73273c34..071763fc 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousBlockingRetrievalFactory.java @@ -15,6 +15,8 @@ package software.amazon.kinesis.retrieval.polling; +import java.time.Duration; +import java.util.function.Function; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -22,13 +24,13 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.retrieval.DataFetcherProviderConfig; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; +import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RecordsFetcherFactory; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; -import java.time.Duration; - /** * */ @@ -42,32 +44,71 @@ public class SynchronousBlockingRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; @NonNull private final RecordsFetcherFactory recordsFetcherFactory; - // private final long listShardsBackoffTimeInMillis; - // private final int maxListShardsRetryAttempts; + private final int maxRecords; private final Duration kinesisRequestTimeout; - public SynchronousBlockingRetrievalFactory(String streamName, KinesisAsyncClient kinesisClient, RecordsFetcherFactory recordsFetcherFactory, int maxRecords, Duration kinesisRequestTimeout) { + private final Function dataFetcherProvider; + + @Deprecated + public SynchronousBlockingRetrievalFactory(String streamName, + KinesisAsyncClient kinesisClient, + RecordsFetcherFactory recordsFetcherFactory, + int maxRecords, + Duration kinesisRequestTimeout) { + this(streamName, + kinesisClient, + recordsFetcherFactory, + maxRecords, + kinesisRequestTimeout, + defaultDataFetcherProvider(kinesisClient)); + } + + public SynchronousBlockingRetrievalFactory(String streamName, + KinesisAsyncClient kinesisClient, + RecordsFetcherFactory recordsFetcherFactory, + int maxRecords, + Duration kinesisRequestTimeout, + Function dataFetcherProvider) { this.streamName = streamName; this.kinesisClient = kinesisClient; this.recordsFetcherFactory = recordsFetcherFactory; this.maxRecords = maxRecords; this.kinesisRequestTimeout = kinesisRequestTimeout; + this.dataFetcherProvider = dataFetcherProvider == null ? + defaultDataFetcherProvider(kinesisClient) : dataFetcherProvider; } @Deprecated - public SynchronousBlockingRetrievalFactory(String streamName, KinesisAsyncClient kinesisClient, RecordsFetcherFactory recordsFetcherFactory, int maxRecords) { + public SynchronousBlockingRetrievalFactory(String streamName, + KinesisAsyncClient kinesisClient, + RecordsFetcherFactory recordsFetcherFactory, + int maxRecords) { this(streamName, kinesisClient, recordsFetcherFactory, maxRecords, PollingConfig.DEFAULT_REQUEST_TIMEOUT); } + private static Function defaultDataFetcherProvider( + KinesisAsyncClient kinesisClient) { + return dataFetcherProviderConfig -> new KinesisDataFetcher(kinesisClient, dataFetcherProviderConfig); + } + @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(@NonNull final ShardInfo shardInfo, - @NonNull final MetricsFactory metricsFactory) { + @NonNull final MetricsFactory metricsFactory) { final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ? StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) : StreamIdentifier.singleStreamInstance(streamName); - return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, streamIdentifier, shardInfo.shardId(), maxRecords, metricsFactory, kinesisRequestTimeout)); + + final DataFetcherProviderConfig kinesisDataFetcherProviderConfig = new KinesisDataFetcherProviderConfig( + streamIdentifier, + shardInfo.shardId(), + metricsFactory, + maxRecords, + kinesisRequestTimeout); + + final DataFetcher dataFetcher = this.dataFetcherProvider.apply(kinesisDataFetcherProviderConfig); + + return new SynchronousGetRecordsRetrievalStrategy(dataFetcher); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java index c6fa619b..7f3b54d5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousGetRecordsRetrievalStrategy.java @@ -14,6 +14,7 @@ */ package software.amazon.kinesis.retrieval.polling; +import java.util.Optional; import lombok.Data; import lombok.NonNull; import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; @@ -26,8 +27,9 @@ import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; @Data @KinesisClientInternalApi public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrievalStrategy { + @NonNull - private final KinesisDataFetcher dataFetcher; + private final DataFetcher dataFetcher; @Override public GetRecordsResponse getRecords(final int maxRecords) { @@ -45,9 +47,14 @@ public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetriev public boolean isShutdown() { return false; } - + @Override public KinesisDataFetcher getDataFetcher() { + throw new UnsupportedOperationException("Deprecated. Use dataFetcher() to retrieve a dataFetcher"); + } + + @Override + public DataFetcher dataFetcher() { return dataFetcher; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java index 4a8c5250..efa11e70 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/SynchronousPrefetchingRetrievalFactory.java @@ -17,7 +17,6 @@ package software.amazon.kinesis.retrieval.polling; import java.time.Duration; import java.util.concurrent.ExecutorService; - import lombok.NonNull; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -25,6 +24,7 @@ import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.GetRecordsRetrievalStrategy; +import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RecordsFetcherFactory; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalFactory; @@ -71,9 +71,15 @@ public class SynchronousPrefetchingRetrievalFactory implements RetrievalFactory final StreamIdentifier streamIdentifier = shardInfo.streamIdentifierSerOpt().isPresent() ? StreamIdentifier.multiStreamInstance(shardInfo.streamIdentifierSerOpt().get()) : StreamIdentifier.singleStreamInstance(streamName); + return new SynchronousGetRecordsRetrievalStrategy( - new KinesisDataFetcher(kinesisClient, streamIdentifier, shardInfo.shardId(), - maxRecords, metricsFactory, maxFutureWait)); + new KinesisDataFetcher(kinesisClient, new KinesisDataFetcherProviderConfig( + streamIdentifier, + shardInfo.shardId(), + metricsFactory, + maxRecords, + maxFutureWait + ))); } @Override 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 f7051ec4..281d738c 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 @@ -107,7 +107,7 @@ public class PrefetchRecordsPublisherTest { @Mock private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @Mock - private KinesisDataFetcher dataFetcher; + private DataFetcher dataFetcher; @Mock private InitialPositionInStreamExtended initialPosition; @Mock @@ -124,7 +124,7 @@ public class PrefetchRecordsPublisherTest { @Before public void setup() { - when(getRecordsRetrievalStrategy.getDataFetcher()).thenReturn(dataFetcher); + when(getRecordsRetrievalStrategy.dataFetcher()).thenReturn(dataFetcher); when(dataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("testStream")); executorService = spy(Executors.newFixedThreadPool(1)); getRecordsCache = new PrefetchRecordsPublisher( diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java index d6d8b6d5..ddc25e21 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/RecordsFetcherFactoryTest.java @@ -40,14 +40,14 @@ public class RecordsFetcherFactoryTest { @Mock private MetricsFactory metricsFactory; @Mock - private KinesisDataFetcher kinesisDataFetcher; + private DataFetcher dataFetcher; @Before public void setUp() { MockitoAnnotations.initMocks(this); recordsFetcherFactory = new SimpleRecordsFetcherFactory(); - when(getRecordsRetrievalStrategy.getDataFetcher()).thenReturn(kinesisDataFetcher); - when(kinesisDataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("stream")); + when(getRecordsRetrievalStrategy.dataFetcher()).thenReturn(dataFetcher); + when(dataFetcher.getStreamIdentifier()).thenReturn(StreamIdentifier.singleStreamInstance("stream")); } @Test @@ -66,5 +66,4 @@ public class RecordsFetcherFactoryTest { metricsFactory, 1); assertThat(recordsCache, instanceOf(PrefetchRecordsPublisher.class)); } - } From 9f639b2342a0f3b0ea0a9849863bbb2ac4aa413d Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 15 Apr 2020 05:24:18 -0400 Subject: [PATCH 063/159] Adding query support for multistream --- .../leases/HierarchicalShardSyncer.java | 26 +-------------- .../amazon/kinesis/leases/LeaseRefresher.java | 13 ++++++++ .../dynamodb/DynamoDBLeaseRefresher.java | 32 +++++++++++++++++-- .../ExceptionThrowingLeaseRefresher.java | 9 ++++++ .../leases/HierarchicalShardSyncerTest.java | 4 +-- ...DynamoDBLeaseRefresherIntegrationTest.java | 2 +- pom.xml | 2 +- 7 files changed, 56 insertions(+), 32 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 5cd6f472..debb89bb 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 @@ -142,8 +142,7 @@ public class HierarchicalShardSyncer { assertAllParentShardsAreClosed(inconsistentShardIds); } final List currentLeases = isMultiStreamMode ? - getLeasesForStream(shardDetector.streamIdentifier(), leaseRefresher) : - leaseRefresher.listLeases(); + leaseRefresher.listLeasesForStream(shardDetector.streamIdentifier()) : leaseRefresher.listLeases(); final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, shardDetector.streamIdentifier()); final LeaseSynchronizer leaseSynchronizer = isLeaseTableEmpty ? new EmptyLeaseTableSynchronizer() : new NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); @@ -171,29 +170,6 @@ public class HierarchicalShardSyncer { } } - // CHECKSTYLE:ON CyclomaticComplexity - - /** Note: This method has package level access solely for testing purposes. - * - * @param streamIdentifier We'll use this stream identifier to filter leases - * @param leaseRefresher Used to fetch leases - * @return Return list of leases (corresponding to shards) of the specified stream. - * @throws DependencyException - * @throws InvalidStateException - * @throws ProvisionedThroughputException - */ - static List getLeasesForStream(StreamIdentifier streamIdentifier, - LeaseRefresher leaseRefresher) - throws DependencyException, ProvisionedThroughputException, InvalidStateException { - List streamLeases = new ArrayList<>(); - for (Lease lease : leaseRefresher.listLeases()) { - if (streamIdentifier.serialize().equals(((MultiStreamLease)lease).streamIdentifier())) { - streamLeases.add(lease); - } - } - return streamLeases; - } - /** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls * and a reshard operation. * @param inconsistentShardIds diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java index 3ba22c2b..f45c4cc2 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java @@ -16,6 +16,7 @@ package software.amazon.kinesis.leases; import java.util.List; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -60,6 +61,18 @@ public interface LeaseRefresher { */ boolean waitUntilLeaseTableExists(long secondsBetweenPolls, long timeoutSeconds) throws DependencyException; + /** + * List all leases for a given stream synchronously. + * + * @throws DependencyException if DynamoDB scan fails in an unexpected way + * @throws InvalidStateException if lease table does not exist + * @throws ProvisionedThroughputException if DynamoDB scan fails due to lack of capacity + * + * @return list of leases + */ + List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, + ProvisionedThroughputException; + /** * List all objects in table synchronously. * 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 1c464afe..c5bb1f66 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 @@ -22,6 +22,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import com.google.common.collect.ImmutableMap; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; @@ -29,6 +30,7 @@ import software.amazon.awssdk.services.dynamodb.model.*; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; @@ -58,6 +60,9 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { private boolean newTableCreated = false; + private static final String STREAM_NAME = "streamName"; + private static final String DDB_STREAM_NAME = ":streamName"; + /** * Constructor. * @@ -263,12 +268,21 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { return System.currentTimeMillis() - startTime; } + /** + * {@inheritDoc} + */ + @Override + public List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, + InvalidStateException, ProvisionedThroughputException { + return list( null, streamIdentifier); + } + /** * {@inheritDoc} */ @Override public List listLeases() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(null); + return list(null, null); } /** @@ -277,22 +291,34 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(1).isEmpty(); + return list(1, null).isEmpty(); } /** * List with the given page size. Package access for integration testing. * * @param limit number of items to consider at a time - used by integration tests to force paging. + * @param streamIdentifier streamIdentifier for multi-stream mode. Can be null. * @return list of leases * @throws InvalidStateException if table does not exist * @throws DependencyException if DynamoDB scan fail in an unexpected way * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity */ - List list(Integer limit) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + List list(Integer limit, StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, + ProvisionedThroughputException { + log.debug("Listing leases from table {}", table); ScanRequest.Builder scanRequestBuilder = ScanRequest.builder().tableName(table); + + if (streamIdentifier != null) { + final Map expressionAttributeValues = ImmutableMap.of( + DDB_STREAM_NAME, AttributeValue.builder().s(streamIdentifier.serialize()).build() + ); + scanRequestBuilder = scanRequestBuilder.filterExpression(STREAM_NAME + " = " + DDB_STREAM_NAME) + .expressionAttributeValues(expressionAttributeValues); + } + if (limit != null) { scanRequestBuilder = scanRequestBuilder.limit(limit); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java index 233ce724..d9d7d01e 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java @@ -19,6 +19,7 @@ import java.util.List; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -54,6 +55,7 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { DELETELEASE(9), DELETEALL(10), UPDATELEASE(11), + LISTLEASESFORSTREAM(12), NONE(Integer.MIN_VALUE); private Integer index; @@ -129,6 +131,13 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { return leaseRefresher.waitUntilLeaseTableExists(secondsBetweenPolls, timeoutSeconds); } + @Override + public List listLeasesForStream(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throwExceptions("listLeasesForStream", ExceptionThrowingLeaseRefresherMethods.LISTLEASESFORSTREAM); + + return leaseRefresher.listLeasesForStream(streamIdentifier); + } + @Override public List listLeases() throws DependencyException, InvalidStateException, ProvisionedThroughputException { 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 374da4cd..44f6acf4 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 @@ -1039,7 +1039,7 @@ public class HierarchicalShardSyncerTest { final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(leases); + when(dynamoDBLeaseRefresher.listLeasesForStream(any(StreamIdentifier.class))).thenReturn(leases); doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); setupMultiStream(); hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, @@ -1049,7 +1049,7 @@ public class HierarchicalShardSyncerTest { assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); verify(shardDetector, times(2)).listShards(); - verify(dynamoDBLeaseRefresher).listLeases(); + verify(dynamoDBLeaseRefresher).listLeasesForStream(any(StreamIdentifier.class)); verify(dynamoDBLeaseRefresher).deleteLease(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java index 414f7975..c9df9106 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 @@ -71,7 +71,7 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest Collection expected = builder.build().values(); // The / 3 here ensures that we will test Dynamo's paging mechanics. - List actual = leaseRefresher.list(numRecordsToPut / 3); + List actual = leaseRefresher.list(numRecordsToPut / 3, null); for (Lease lease : actual) { assertNotNull(expected.remove(lease)); diff --git a/pom.xml b/pom.xml index f21b230e..4e8da722 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.11.8-SNAPSHOT + 2.10.65-SNAPSHOT From 297340124818f289ceec32ae49ac869517f016eb Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 21 Apr 2020 14:31:07 -0400 Subject: [PATCH 064/159] reverting pom changes --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4e8da722..f21b230e 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.10.65-SNAPSHOT + 2.11.8-SNAPSHOT From 054cd8828451f3349115ab075de88155dcc27de3 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 15 Apr 2020 17:57:38 -0700 Subject: [PATCH 065/159] ShardEnd sync persistence change quick fix Addressing comments --- .../software/amazon/kinesis/leases/Lease.java | 26 +++++++++++++++++-- .../dynamodb/DynamoDBLeaseSerializer.java | 11 ++++++++ .../kinesis/lifecycle/ShutdownTask.java | 21 +++++++++++++++ .../amazon/kinesis/leases/LeaseBuilder.java | 3 ++- ...DynamoDBLeaseRefresherIntegrationTest.java | 13 ++++++++++ .../leases/dynamodb/TestHarnessBuilder.java | 2 ++ .../kinesis/lifecycle/ShutdownTaskTest.java | 15 ++++++++++- 7 files changed, 87 insertions(+), 4 deletions(-) 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 682a6f9e..a04e2725 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 @@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit; @NoArgsConstructor @Getter @Accessors(fluent = true) -@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos"}) +@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "childShardIds"}) @ToString public class Lease { /* @@ -87,6 +87,7 @@ public class Lease { */ private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); /** * Copy constructor, used by clone(). @@ -96,13 +97,21 @@ public class Lease { protected Lease(Lease lease) { this(lease.leaseKey(), lease.leaseOwner(), lease.leaseCounter(), lease.concurrencyToken(), lease.lastCounterIncrementNanos(), lease.checkpoint(), lease.pendingCheckpoint(), - lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds()); + lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.childShardIds()); } public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, final UUID concurrencyToken, final Long lastCounterIncrementNanos, final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds) { + this(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, pendingCheckpoint, + ownerSwitchesSinceCheckpoint, parentShardIds, new HashSet<>()); + } + + public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, + final UUID concurrencyToken, final Long lastCounterIncrementNanos, + final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, + final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds, final Set childShardIds) { this.leaseKey = leaseKey; this.leaseOwner = leaseOwner; this.leaseCounter = leaseCounter; @@ -114,6 +123,9 @@ public class Lease { if (parentShardIds != null) { this.parentShardIds.addAll(parentShardIds); } + if (childShardIds != null) { + this.childShardIds.addAll(childShardIds); + } } /** @@ -134,6 +146,7 @@ public class Lease { checkpoint(lease.checkpoint); pendingCheckpoint(lease.pendingCheckpoint); parentShardIds(lease.parentShardIds); + childShardIds(lease.childShardIds()); } /** @@ -231,6 +244,15 @@ public class Lease { this.parentShardIds.addAll(parentShardIds); } + /** + * Sets childShardIds. + * + * @param childShardIds may not be null + */ + public void childShardIds(@NonNull final Collection childShardIds) { + this.childShardIds.addAll(childShardIds); + } + /** * Sets leaseOwner. * 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 a02e2a6e..52b4d014 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java @@ -30,6 +30,7 @@ import software.amazon.awssdk.services.dynamodb.model.ExpectedAttributeValue; import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; import software.amazon.awssdk.services.dynamodb.model.KeyType; import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.leases.DynamoUtils; import software.amazon.kinesis.leases.Lease; @@ -51,6 +52,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String PENDING_CHECKPOINT_SEQUENCE_KEY = "pendingCheckpoint"; private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; + private static final String CHILD_SHARD_ID_KEY = "childShardId"; @Override public Map toDynamoRecord(final Lease lease) { @@ -69,6 +71,9 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { if (lease.parentShardIds() != null && !lease.parentShardIds().isEmpty()) { result.put(PARENT_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.parentShardIds())); } + if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { + result.put(CHILD_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.childShardIds())); + } if (lease.pendingCheckpoint() != null && !lease.pendingCheckpoint().sequenceNumber().isEmpty()) { result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.pendingCheckpoint().sequenceNumber())); @@ -97,6 +102,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { DynamoUtils.safeGetLong(dynamoRecord, CHECKPOINT_SUBSEQUENCE_NUMBER_KEY)) ); leaseToUpdate.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY)); + leaseToUpdate.childShardIds(DynamoUtils.safeGetSS(dynamoRecord, CHILD_SHARD_ID_KEY)); if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) { leaseToUpdate.pendingCheckpoint( @@ -220,6 +226,11 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, AttributeValueUpdate.builder().action(AttributeAction.DELETE).build()); } + + if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { + result.put(CHILD_SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); + } + return result; } 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 33eb4497..96b17b3c 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 @@ -45,7 +45,10 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.UUID; import java.util.function.Function; /** @@ -109,8 +112,13 @@ public class ShutdownTask implements ConsumerTask { final long startTime = System.currentTimeMillis(); if (reason == ShutdownReason.SHARD_END) { // Create new lease for the child shards if they don't exist. + // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. + // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. + // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. + // This scenario could happen when customer deletes the stream while leaving the KCL application running. if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); + updateLeasesForChildShards(); } else { log.warn("Shard {} no longer exists. Shutting down consumer with SHARD_END reason without creating leases for child shards.", shardInfoIdProvider.apply(shardInfo)); } @@ -182,6 +190,19 @@ public class ShutdownTask implements ConsumerTask { } } + private void updateLeasesForChildShards() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + final Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfoIdProvider.apply(shardInfo)); + Set childShardIds = new HashSet<>(); + for (ChildShard childShard : childShards) { + childShardIds.add(childShard.shardId()); + } + + final Lease upatedLease = currentLease.copy(); + upatedLease.childShardIds(childShardIds); + leaseCoordinator.updateLease(upatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, shardInfoIdProvider.apply(shardInfo)); + } + /* * (non-Javadoc) * diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java index ee38116f..8ab99a18 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java @@ -34,9 +34,10 @@ public class LeaseBuilder { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); public Lease build() { return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds); + checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds); } } \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java index 414f7975..99dcd64d 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 @@ -25,6 +25,7 @@ import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.verify; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -239,6 +240,18 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest assertNull(newLease); } + @Test + public void testUpdateLease() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + Lease lease = builder.withLease("1").build().get("1"); + Lease updatedLease = lease.copy(); + updatedLease.childShardIds(Collections.singleton("updatedChildShardId")); + + leaseRefresher.updateLease(updatedLease); + Lease newLease = leaseRefresher.getLease(lease.leaseKey()); + assertEquals(Collections.singleton("updatedChildShardId"), newLease.childShardIds()); + } + /** * Tests deleteLease when a lease does not exist. */ diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java index 6afa0045..677303d6 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/TestHarnessBuilder.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertNotNull; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; @@ -72,6 +73,7 @@ public class TestHarnessBuilder { lease.leaseCounter(0L); lease.leaseOwner(owner); lease.parentShardIds(Collections.singleton("parentShardId")); + lease.childShardIds(new HashSet<>()); lease.leaseKey(shardId); return lease; diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java index cbb9b834..9f1737fe 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 @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.UUID; import org.junit.Before; import org.junit.Test; @@ -77,7 +78,7 @@ public class ShutdownTaskTest { private static final ShutdownReason LEASE_LOST_SHUTDOWN_REASON = ShutdownReason.LEASE_LOST; private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory(); - private final String concurrencyToken = "testToken4398"; + private final String concurrencyToken = "0-1-2-3-4"; private final String shardId = "shardId-0"; private boolean cleanupLeasesOfCompletedShards = false; private boolean ignoreUnexpectedChildShards = false; @@ -122,6 +123,7 @@ public class ShutdownTaskTest { @Test public final void testCallWhenApplicationDoesNotCheckpoint() { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(createLease()); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); @@ -162,6 +164,7 @@ public class ShutdownTaskTest { hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards()); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(createLease()); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); @@ -169,6 +172,7 @@ public class ShutdownTaskTest { verify(recordsPublisher).shutdown(); verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator).updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); verify(leaseRefresher, times(2)).createLeaseIfNotExists(Matchers.any(Lease.class)); verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } @@ -249,4 +253,13 @@ public class ShutdownTaskTest { return childShards; } + private Lease createLease() { + Lease lease = new Lease(); + lease.leaseKey("shardId-0"); + lease.leaseOwner("leaseOwner"); + lease.parentShardIds(Collections.singleton("parentShardIds")); + + return lease; + } + } From 93dac82bd610460e3c67e96da799d3763911262b Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 22 Apr 2020 12:35:23 -0700 Subject: [PATCH 066/159] Fixing unit tests --- .../amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java | 2 +- .../kinesis/retrieval/polling/PrefetchRecordsPublisher.java | 2 +- .../amazon/kinesis/leases/HierarchicalShardSyncerTest.java | 2 +- .../kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) 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 20f5b39f..c24ec618 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 @@ -53,7 +53,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber"; private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; - private static final String CHILD_SHARD_ID_KEY = "childShardId"; + private static final String CHILD_SHARD_ID_KEY = "childShardIds"; @Override public Map toDynamoRecord(final Lease lease) { 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 d9e00669..4a0c75eb 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 @@ -433,7 +433,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { .records(records) .millisBehindLatest(getRecordsResult.millisBehindLatest()) .cacheEntryTime(lastSuccessfulCall) - .isAtShardEnd(getRecordsRetrievalStrategy.getDataFetcher().isShardEndReached()) + .isAtShardEnd(getRecordsRetrievalStrategy.dataFetcher().isShardEndReached()) .childShards(getRecordsResult.childShards()) .build(); 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 44f6acf4..cbf7add7 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 @@ -969,7 +969,7 @@ public class HierarchicalShardSyncerTest { parentShardIds.add(shard.adjacentParentShardId()); } return new Lease(shard.shardId(), leaseOwner, 0L, UUID.randomUUID(), 0L, checkpoint, null, 0L, - parentShardIds, null); + parentShardIds, new HashSet<>(), null); }).collect(Collectors.toList()); } 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 f22e6e4d..1daf85b8 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 @@ -55,7 +55,7 @@ public class DynamoDBLeaseRenewerTest { private LeaseRefresher leaseRefresher; private static Lease newLease(String leaseKey) { - return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>(), null); + return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>(), new HashSet<>(), null); } @Before From cde09ca191b06f25d45fa723a7d6aed2797e0d92 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 22 Apr 2020 15:00:10 -0700 Subject: [PATCH 067/159] Address comments --- .../leases/dynamodb/DynamoDBLeaseSerializer.java | 8 ++++---- .../amazon/kinesis/lifecycle/ShutdownTask.java | 12 +++++------- .../kinesis/lifecycle/ShutdownTaskTest.java | 15 +++++++++------ 3 files changed, 18 insertions(+), 17 deletions(-) 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 c24ec618..85632500 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 @@ -53,7 +53,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber"; private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; - private static final String CHILD_SHARD_ID_KEY = "childShardIds"; + private static final String CHILD_SHARD_IDS_KEY = "childShardIds"; @Override public Map toDynamoRecord(final Lease lease) { @@ -73,7 +73,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PARENT_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.parentShardIds())); } if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { - result.put(CHILD_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.childShardIds())); + result.put(CHILD_SHARD_IDS_KEY, DynamoUtils.createAttributeValue(lease.childShardIds())); } if (lease.pendingCheckpoint() != null && !lease.pendingCheckpoint().sequenceNumber().isEmpty()) { @@ -107,7 +107,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { DynamoUtils.safeGetLong(dynamoRecord, CHECKPOINT_SUBSEQUENCE_NUMBER_KEY)) ); leaseToUpdate.parentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY)); - leaseToUpdate.childShardIds(DynamoUtils.safeGetSS(dynamoRecord, CHILD_SHARD_ID_KEY)); + leaseToUpdate.childShardIds(DynamoUtils.safeGetSS(dynamoRecord, CHILD_SHARD_IDS_KEY)); if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) { leaseToUpdate.pendingCheckpoint( @@ -243,7 +243,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { - result.put(CHILD_SHARD_ID_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); + result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); } return result; 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 9fb822a5..bf07f6e2 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 @@ -52,6 +52,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; import java.util.function.Function; +import java.util.stream.Collectors; /** * Task for invoking the ShardRecordProcessor shutdown() callback. @@ -195,14 +196,11 @@ public class ShutdownTask implements ConsumerTask { private void updateLeasesForChildShards() throws DependencyException, InvalidStateException, ProvisionedThroughputException { final Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfoIdProvider.apply(shardInfo)); - Set childShardIds = new HashSet<>(); - for (ChildShard childShard : childShards) { - childShardIds.add(childShard.shardId()); - } + Set childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet()); - final Lease upatedLease = currentLease.copy(); - upatedLease.childShardIds(childShardIds); - leaseCoordinator.updateLease(upatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, shardInfoIdProvider.apply(shardInfo)); + final Lease updatedLease = currentLease.copy(); + updatedLease.childShardIds(childShardIds); + leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, shardInfoIdProvider.apply(shardInfo)); } /* 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 9f1737fe..c94a3266 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 @@ -28,6 +28,7 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.UUID; @@ -123,7 +124,8 @@ public class ShutdownTaskTest { @Test public final void testCallWhenApplicationDoesNotCheckpoint() { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); - when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(createLease()); + Lease heldLease = createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId")); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); @@ -164,7 +166,8 @@ public class ShutdownTaskTest { hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards()); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(createLease()); + Lease heldLease = createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId")); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); final TaskResult result = task.call(); @@ -253,11 +256,11 @@ public class ShutdownTaskTest { return childShards; } - private Lease createLease() { + private Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { Lease lease = new Lease(); - lease.leaseKey("shardId-0"); - lease.leaseOwner("leaseOwner"); - lease.parentShardIds(Collections.singleton("parentShardIds")); + lease.leaseKey(leaseKey); + lease.leaseOwner(leaseOwner); + lease.parentShardIds(parentShardIds); return lease; } From f51657f6f7c476b73ddbfea04844aae1c932e5c5 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 17 Apr 2020 10:37:19 -0700 Subject: [PATCH 068/159] Changes for adding stream as dimension in multistream mode. Changes for deferred cleanup of stale stream leases to free up record processors. --- .../amazon/kinesis/coordinator/Scheduler.java | 142 ++++++++++++++---- .../leases/dynamodb/DynamoDBLeaseRenewer.java | 6 + .../amazon/kinesis/lifecycle/ProcessTask.java | 5 + .../amazon/kinesis/metrics/MetricsUtil.java | 7 + .../retrieval/polling/KinesisDataFetcher.java | 4 +- .../polling/PrefetchRecordsPublisher.java | 7 +- .../kinesis/coordinator/SchedulerTest.java | 81 +++++++++- 7 files changed, 221 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 4d6c1fd7..478080d0 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 @@ -19,6 +19,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import io.reactivex.plugins.RxJavaPlugins; +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -40,6 +43,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import lombok.AccessLevel; +import lombok.Data; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.NonNull; @@ -103,6 +107,7 @@ public class Scheduler implements Runnable { private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; private static final long HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS = 5000L; private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; + private static final long OLD_STREAM_DEFERRED_DELETION_PERIOD_MILLIS = 1 * 60 * 60 * 1000L; private SchedulerLog slog = new SchedulerLog(); @@ -138,7 +143,6 @@ public class Scheduler implements Runnable { private final long failoverTimeMillis; private final long taskBackoffTimeMillis; private final boolean isMultiStreamMode; - // TODO : halo : make sure we generate streamConfig if entry not present. private final Map currentStreamConfigMap; private MultiStreamTracker multiStreamTracker; private final long listShardsBackoffTimeMillis; @@ -150,6 +154,7 @@ public class Scheduler implements Runnable { private final HierarchicalShardSyncer hierarchicalShardSyncer; private final long schedulerInitializationBackoffTimeMillis; private final LeaderDecider leaderDecider; + private final Map staleStreamDeletionMap = new HashMap<>(); // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. @@ -239,8 +244,6 @@ public class Scheduler implements Runnable { this.executorService = this.coordinatorConfig.coordinatorFactory().createExecutorService(); this.diagnosticEventFactory = diagnosticEventFactory; this.diagnosticEventHandler = new DiagnosticEventLogger(); - // TODO : Halo : Handle case of no StreamConfig present in streamConfigList() for the supplied streamName. - // TODO : Pass the immutable map here instead of using mst.streamConfigList() this.shardSyncTaskManagerProvider = streamConfig -> this.leaseManagementConfig .leaseManagementFactory(leaseSerializer, isMultiStreamMode) .createShardSyncTaskManager(this.metricsFactory, streamConfig); @@ -461,9 +464,16 @@ public class Scheduler implements Runnable { newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); - // This is done to ensure that we clean up the stale streams lingering in the lease table. - syncStreamsFromLeaseTableOnAppInit(); + List leases; + // This is done to ensure that we clean up the stale streams lingering in the lease table. + if (!leasesSyncedOnAppInit && isMultiStreamMode) { + leases = fetchMultiStreamLeases(); + syncStreamsFromLeaseTableOnAppInit(leases); + leasesSyncedOnAppInit = true; + } + + // For new streams discovered, do a shard sync and update the currentStreamConfigMap for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); @@ -478,41 +488,117 @@ public class Scheduler implements Runnable { } } - // TODO: Remove assumption that each Worker gets the full list of streams + // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion. + // It is assumed that all the workers will always have the latest and consistent snapshot of streams + // from the multiStreamTracker. + // + // The following streams transition state among two workers are NOT considered safe, where Worker 2, on + // initialization learn about D from lease table and delete the leases for D, as it is not available + // in its latest MultiStreamTracker. + // Worker 1 : A,B,C -> A,B,C,D (latest) + // Worker 2 : BOOTS_UP -> A,B,C (stale) + // + // The following streams transition state among two workers are NOT considered safe, where Worker 2 might + // end up deleting the leases for A and D and loose progress made so far. + // Worker 1 : A,B,C -> A,B,C,D (latest) + // Worker 2 : A,B,C -> B,C (stale/partial) + // + // In order to give workers with stale stream info, sufficient time to learn about the new streams + // before attempting to delete it, we will be deferring the leases deletion based on the + // defer time period. + Iterator currentStreamConfigIter = currentStreamConfigMap.keySet().iterator(); while (currentStreamConfigIter.hasNext()) { StreamIdentifier streamIdentifier = currentStreamConfigIter.next(); if (!newStreamConfigMap.containsKey(streamIdentifier)) { - log.info("Found old/deleted stream: " + streamIdentifier + ". Syncing shards of that stream."); - ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(currentStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.syncShardAndLeaseInfo(); - currentStreamConfigIter.remove(); - streamsSynced.add(streamIdentifier); + staleStreamDeletionMap.putIfAbsent(streamIdentifier, Instant.now()); } } + + // Now let's scan the streamIdentifiers eligible for deferred deletion and delete them. + // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and + // the streamIdentifiers are not present in the latest snapshot. + final Set streamIdsToBeDeleted = staleStreamDeletionMap.keySet().stream() + .filter(streamIdentifier -> + Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() + >= getOldStreamDeferredDeletionPeriodMillis() && + !newStreamConfigMap.containsKey(streamIdentifier)) + .collect(Collectors.toSet()); + + streamsSynced.addAll(deleteMultiStreamLeases(streamIdsToBeDeleted)); + streamSyncWatch.reset().start(); } return streamsSynced; } + @VisibleForTesting + long getOldStreamDeferredDeletionPeriodMillis() { + return OLD_STREAM_DEFERRED_DELETION_PERIOD_MILLIS; + } + @VisibleForTesting boolean shouldSyncStreamsNow() { return isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); } - private void syncStreamsFromLeaseTableOnAppInit() + private void syncStreamsFromLeaseTableOnAppInit(List leases) { + final Set streamIdentifiers = leases.stream() + .map(lease -> StreamIdentifier.multiStreamInstance(lease.streamIdentifier())) + .collect(Collectors.toSet()); + for (StreamIdentifier streamIdentifier : streamIdentifiers) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + currentStreamConfigMap.put(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + } + } + } + + private List fetchMultiStreamLeases() throws DependencyException, ProvisionedThroughputException, InvalidStateException { - if (!leasesSyncedOnAppInit && isMultiStreamMode) { - final Set streamIdentifiers = leaseCoordinator.leaseRefresher().listLeases().stream() - .map(lease -> StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())) - .collect(Collectors.toSet()); - for (StreamIdentifier streamIdentifier : streamIdentifiers) { - if (!currentStreamConfigMap.containsKey(streamIdentifier)) { - currentStreamConfigMap.put(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); + return (List) ((List) leaseCoordinator.leaseRefresher().listLeases()); + } + + private Set deleteMultiStreamLeases(Set streamIdentifiers) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final Set streamsSynced = new HashSet<>(); + List leases = null; + Map> streamIdToShardsMap = null; + for(StreamIdentifier streamIdentifier : streamIdentifiers) { + if (leases == null) { + // Lazy Load once and use many times for this iteration. + leases = fetchMultiStreamLeases(); + } + if (streamIdToShardsMap == null) { + // Lazy load once and use many times for this iteration. + streamIdToShardsMap = leases.stream().collect(Collectors + .groupingBy(MultiStreamLease::streamIdentifier, + Collectors.toCollection(ArrayList::new))); + } + log.warn("Found old/deleted stream: " + streamIdentifier + ". Deleting leases of this stream."); + // Deleting leases will cause the workers to shutdown the record processors for these shards. + if (deleteMultiStreamLeases(streamIdToShardsMap.get(streamIdentifier.serialize()))) { + currentStreamConfigMap.remove(streamIdentifier); + staleStreamDeletionMap.remove(streamIdentifier); + streamsSynced.add(streamIdentifier); + } + } + return streamsSynced; + } + + private boolean deleteMultiStreamLeases(List leases) { + if (leases != null) { + for (MultiStreamLease lease : leases) { + try { + leaseRefresher.deleteLease(lease); + } catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) { + log.error( + "Unable to delete stale stream lease {}. Skipping further deletions for this stream. Will retry later.", + lease.leaseKey(), e); + return false; } } - leasesSyncedOnAppInit = true; } + return true; } // When a stream is no longer needed to be tracked, return a default StreamConfig with LATEST for faster shard end. @@ -549,6 +635,7 @@ public class Scheduler implements Runnable { * Requests a graceful shutdown of the worker, notifying record processors, that implement * {@link ShutdownNotificationAware}, of the impending shutdown. This gives the record processor a final chance to * checkpoint. + * * This will only create a single shutdown future. Additional attempts to start a graceful shutdown will return the * previous future. * @@ -575,8 +662,8 @@ public class Scheduler implements Runnable { * * * @return a future that will be set once the shutdown has completed. True indicates that the graceful shutdown - * completed successfully. A false value indicates that a non-exception case caused the shutdown process to - * terminate early. + * completed successfully. A false value indicates that a non-exception case caused the shutdown process to + * terminate early. */ public Future startGracefulShutdown() { synchronized (this) { @@ -593,8 +680,9 @@ public class Scheduler implements Runnable { * shutdowns in your own executor, or execute the shutdown synchronously. * * @return a callable that run the graceful shutdown process. This may return a callable that return true if the - * graceful shutdown has already been completed. - * @throws IllegalStateException thrown by the callable if another callable has already started the shutdown process. + * graceful shutdown has already been completed. + * @throws IllegalStateException + * thrown by the callable if another callable has already started the shutdown process. */ public Callable createGracefulShutdownCallable() { if (shutdownComplete()) { @@ -736,7 +824,8 @@ public class Scheduler implements Runnable { /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. * - * @param shardInfo Kinesis shard info + * @param shardInfo + * Kinesis shard info * @return ShardConsumer for the shard */ ShardConsumer createOrGetShardConsumer(@NonNull final ShardInfo shardInfo, @@ -764,7 +853,7 @@ public class Scheduler implements Runnable { @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory); ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, - checkpoint); + checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt()); @@ -801,6 +890,7 @@ public class Scheduler implements Runnable { /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. + * * This method relies on ShardInfo.equals() method returning true for ShardInfo objects which may have been * instantiated with parentShardIds in a different order (and rest of the fields being the equal). For example * shardInfo1.equals(shardInfo2) should return true with shardInfo1 and shardInfo2 defined as follows. ShardInfo diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java index 04d987a2..a1e0afcc 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java @@ -36,9 +36,11 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseRenewer; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -297,6 +299,10 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation); if (StringUtils.isNotEmpty(shardId)) { + if(lease instanceof MultiStreamLease) { + MetricsUtil.addStreamId(scope, + StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())); + } MetricsUtil.addShardId(scope, shardId); } 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 fd036c9f..f576154a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java @@ -23,6 +23,7 @@ import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; @@ -109,6 +110,8 @@ public class ProcessTask implements ConsumerTask { @Override public TaskResult call() { final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION); + shardInfo.streamIdentifierSerOpt() + .ifPresent(streamId -> MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(streamId))); MetricsUtil.addShardId(scope, shardInfo.shardId()); long startTimeMillis = System.currentTimeMillis(); boolean success = false; @@ -197,6 +200,8 @@ public class ProcessTask implements ConsumerTask { .checkpointer(recordProcessorCheckpointer).millisBehindLatest(input.millisBehindLatest()).build(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PROCESS_TASK_OPERATION); + shardInfo.streamIdentifierSerOpt() + .ifPresent(streamId -> MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(streamId))); MetricsUtil.addShardId(scope, shardInfo.shardId()); final long startTime = System.currentTimeMillis(); try { diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java index c6f2fe6f..a2f9d84b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java @@ -19,6 +19,7 @@ import org.apache.commons.lang3.StringUtils; import lombok.NonNull; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; +import software.amazon.kinesis.common.StreamIdentifier; /** * @@ -26,6 +27,7 @@ import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; public class MetricsUtil { public static final String OPERATION_DIMENSION_NAME = "Operation"; public static final String SHARD_ID_DIMENSION_NAME = "ShardId"; + public static final String STREAM_IDENTIFIER = "StreamId"; private static final String WORKER_IDENTIFIER_DIMENSION = "WorkerIdentifier"; private static final String TIME_METRIC = "Time"; private static final String SUCCESS_METRIC = "Success"; @@ -51,6 +53,11 @@ public class MetricsUtil { addOperation(metricsScope, SHARD_ID_DIMENSION_NAME, shardId); } + public static void addStreamId(@NonNull final MetricsScope metricsScope, @NonNull final StreamIdentifier streamId) { + streamId.accountIdOptional() + .ifPresent(acc -> addOperation(metricsScope, STREAM_IDENTIFIER, streamId.serialize())); + } + public static void addWorkerIdentifier(@NonNull final MetricsScope metricsScope, @NonNull final String workerIdentifier) { addOperation(metricsScope, WORKER_IDENTIFIER_DIMENSION, workerIdentifier); 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 82b31915..0b7df88d 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 @@ -232,6 +232,7 @@ public class KinesisDataFetcher implements DataFetcher { // TODO: Check if this metric is fine to be added final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION); + MetricsUtil.addStreamId(metricsScope, streamIdentifier); MetricsUtil.addShardId(metricsScope, shardId); boolean success = false; long startTime = System.currentTimeMillis(); @@ -315,6 +316,7 @@ public class KinesisDataFetcher implements DataFetcher { GetRecordsRequest request = getGetRecordsRequest(nextIterator); final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, OPERATION); + MetricsUtil.addStreamId(metricsScope, streamIdentifier); MetricsUtil.addShardId(metricsScope, shardId); boolean success = false ; long startTime = System.currentTimeMillis(); @@ -325,7 +327,7 @@ public class KinesisDataFetcher implements DataFetcher { } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } catch (InterruptedException e) { - // TODO: Check behavior + // TODO: Check behaviorF log.debug("{} : Interrupt called on method, shutdown initiated", streamAndShardId); throw new RuntimeException(e); } catch (TimeoutException e) { 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 4a0c75eb..c80c9860 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 @@ -45,6 +45,7 @@ import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.RequestDetails; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsLevel; @@ -88,6 +89,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { private final DefaultGetRecordsCacheDaemon defaultGetRecordsCacheDaemon; private boolean started = false; private final String operation; + private final StreamIdentifier streamId; private final String streamAndShardId; private Subscriber subscriber; @VisibleForTesting @Getter @@ -219,8 +221,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon(); Validate.notEmpty(operation, "Operation cannot be empty"); this.operation = operation; - this.streamAndShardId = - this.getRecordsRetrievalStrategy.dataFetcher().getStreamIdentifier().serialize() + ":" + shardId; + this.streamId = this.getRecordsRetrievalStrategy.dataFetcher().getStreamIdentifier(); + this.streamAndShardId = this.streamId.serialize() + ":" + shardId; } @Override @@ -453,6 +455,7 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { log.info("{} : records threw ExpiredIteratorException - restarting" + " after greatest seqNum passed to customer", streamAndShardId, e); + MetricsUtil.addStreamId(scope, streamId); scope.addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.COUNT, MetricsLevel.SUMMARY); publisherSession.dataFetcher().restartIterator(); 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 1de7b101..96cec37a 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 @@ -36,6 +36,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.atMost; +import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -443,7 +444,7 @@ public class SchedulerTest { } @Test - public final void testMultiStreamOnlyStaleStreamsAreSynced() + public final void testMultiStreamStaleStreamsAreNotDeletedImmediately() throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -462,16 +463,49 @@ public class SchedulerTest { metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedPendingStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(HashSet::new)); + Assert.assertEquals(Sets.newHashSet(), syncedStreams); + Assert.assertEquals(Sets.newHashSet(streamConfigList1), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(expectedPendingStreams, + scheduler.staleStreamDeletionMap().keySet()); + } + + @Test + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + when(scheduler.getOldStreamDeferredDeletionPeriodMillis()).thenReturn(0L); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Set expectedSyncedStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( Collectors.toCollection(HashSet::new)); Assert.assertEquals(expectedSyncedStreams, syncedStreams); Assert.assertEquals(Sets.newHashSet(streamConfigList2), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(Sets.newHashSet(), + scheduler.staleStreamDeletionMap().keySet()); } @Test - public final void testMultiStreamSyncOnlyNewAndStaleStreamsAreSynced() + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately() throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -490,6 +524,47 @@ public class SchedulerTest { metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); + Set expectedSyncedStreams = IntStream.range(5, 7) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + Set expectedPendingStreams = IntStream.range(1, 3) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + List expectedCurrentStreamConfigs = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + Assert.assertEquals(Sets.newHashSet(expectedCurrentStreamConfigs), + Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(expectedPendingStreams, + scheduler.staleStreamDeletionMap().keySet()); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreDeletedAfterDefermentPeriod() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + List streamConfigList2 = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) + .retrievalFactory(retrievalFactory); + when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, + metricsConfig, processorConfig, retrievalConfig)); + when(scheduler.shouldSyncStreamsNow()).thenReturn(true); + when(scheduler.getOldStreamDeferredDeletionPeriodMillis()).thenReturn(0L); + Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Set expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) @@ -497,6 +572,8 @@ public class SchedulerTest { Assert.assertEquals(expectedSyncedStreams, syncedStreams); Assert.assertEquals(Sets.newHashSet(streamConfigList2), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); + Assert.assertEquals(Sets.newHashSet(), + scheduler.staleStreamDeletionMap().keySet()); } @Test From 45387bfd74336a70f2ef066b6bce21cb42c34931 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 17 Apr 2020 17:11:04 -0700 Subject: [PATCH 069/159] Adding logic to garbage collect active streams from stale streams list. --- .../amazon/kinesis/coordinator/Scheduler.java | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 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 478080d0..5a486a0d 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 @@ -43,7 +43,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import lombok.AccessLevel; -import lombok.Data; import lombok.Getter; import lombok.NoArgsConstructor; import lombok.NonNull; @@ -518,14 +517,19 @@ public class Scheduler implements Runnable { // Now let's scan the streamIdentifiers eligible for deferred deletion and delete them. // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and // the streamIdentifiers are not present in the latest snapshot. - final Set streamIdsToBeDeleted = staleStreamDeletionMap.keySet().stream() + final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet() + .stream().collect(Collectors + .partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), + Collectors.toSet())); + final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream() .filter(streamIdentifier -> Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() - >= getOldStreamDeferredDeletionPeriodMillis() && - !newStreamConfigMap.containsKey(streamIdentifier)) - .collect(Collectors.toSet()); + >= getOldStreamDeferredDeletionPeriodMillis()).collect(Collectors.toSet()); + streamsSynced.addAll(deleteMultiStreamLeases(staleStreamIdsToBeDeleted)); - streamsSynced.addAll(deleteMultiStreamLeases(streamIdsToBeDeleted)); + // Purge the active streams from stale streams list. + final Set staleStreamIdsToBeRevived = staleStreamIdDeletionDecisionMap.get(true); + removeActiveStreamsFromStaleStreamsList(staleStreamIdsToBeRevived); streamSyncWatch.reset().start(); } @@ -558,6 +562,12 @@ public class Scheduler implements Runnable { return (List) ((List) leaseCoordinator.leaseRefresher().listLeases()); } + private void removeActiveStreamsFromStaleStreamsList(Set streamIdentifiers) { + for(StreamIdentifier streamIdentifier : streamIdentifiers) { + staleStreamDeletionMap.remove(streamIdentifier); + } + } + private Set deleteMultiStreamLeases(Set streamIdentifiers) throws DependencyException, ProvisionedThroughputException, InvalidStateException { final Set streamsSynced = new HashSet<>(); From 596e3ee79791e4086a448f0319c9ecf1f23e9287 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 21 Apr 2020 21:31:13 -0700 Subject: [PATCH 070/159] Exposing interface method to set the wait time for old stream deletion. Adding license details to new classes --- .../amazon/kinesis/common/RequestDetails.java | 15 +++++++++++ .../amazon/kinesis/common/StreamConfig.java | 15 +++++++++++ .../kinesis/common/StreamIdentifier.java | 15 +++++++++++ .../amazon/kinesis/coordinator/Scheduler.java | 12 +++------ .../kinesis/processor/MultiStreamTracker.java | 25 ++++++++++++++++++- .../retrieval/fanout/FanOutConfig.java | 2 +- .../retrieval/polling/KinesisDataFetcher.java | 2 +- .../kinesis/coordinator/SchedulerTest.java | 10 +++----- 8 files changed, 79 insertions(+), 17 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java index ca14155e..9f511123 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/RequestDetails.java @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.common; import lombok.experimental.Accessors; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamConfig.java index 999182b6..8856a4a0 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 @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.common; import lombok.Value; 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 f4cbac29..7a416c7a 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 @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.common; import com.google.common.base.Joiner; 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 5a486a0d..22be7fc7 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 @@ -106,7 +106,6 @@ public class Scheduler implements Runnable { private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; private static final long HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS = 5000L; private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; - private static final long OLD_STREAM_DEFERRED_DELETION_PERIOD_MILLIS = 1 * 60 * 60 * 1000L; private SchedulerLog slog = new SchedulerLog(); @@ -273,7 +272,7 @@ public class Scheduler implements Runnable { this.shardDetectorProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); - // TODO : Halo : Check if this needs to be per stream. + // TODO : LTR : Check if this needs to be per stream. this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(isMultiStreamMode); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager( @@ -459,6 +458,8 @@ public class Scheduler implements Runnable { if (shouldSyncStreamsNow()) { final Map newStreamConfigMap = new HashMap<>(); + final long waitPeriodToDeleteOldStreamsMillis = multiStreamTracker.waitPeriodToDeleteOldStreams() + .toMillis(); // Making an immutable copy newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); @@ -524,7 +525,7 @@ public class Scheduler implements Runnable { final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream() .filter(streamIdentifier -> Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() - >= getOldStreamDeferredDeletionPeriodMillis()).collect(Collectors.toSet()); + >= waitPeriodToDeleteOldStreamsMillis).collect(Collectors.toSet()); streamsSynced.addAll(deleteMultiStreamLeases(staleStreamIdsToBeDeleted)); // Purge the active streams from stale streams list. @@ -536,11 +537,6 @@ public class Scheduler implements Runnable { return streamsSynced; } - @VisibleForTesting - long getOldStreamDeferredDeletionPeriodMillis() { - return OLD_STREAM_DEFERRED_DELETION_PERIOD_MILLIS; - } - @VisibleForTesting boolean shouldSyncStreamsNow() { return isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); 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 171687bc..785778db 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 @@ -1,20 +1,43 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.processor; import software.amazon.kinesis.common.StreamConfig; +import java.time.Duration; import java.util.List; -import java.util.Map; /** * Interface for stream trackers. This is useful for KCL Workers that need * to consume data from multiple streams. + * KCL will periodically probe this interface to learn about the new and old streams. */ public interface MultiStreamTracker { /** * Returns the list of stream config, to be processed by the current application. + * Note that this method will be called periodically called by the KCL to learn about the new and old streams. * * @return List of StreamConfig */ List streamConfigList(); + + /** + * Duration to wait before deleting the old streams in the lease table. + * @return Wait time before deleting old streams + */ + Duration waitPeriodToDeleteOldStreams(); } 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 cbbcb483..fafe7e18 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 @@ -85,7 +85,7 @@ public class FanOutConfig implements RetrievalSpecificConfig { return new FanOutRetrievalFactory(kinesisClient, streamName, this::getOrCreateConsumerArn); } - // TODO : Halo. Need Stream Specific ConsumerArn to be passed from Customer + // TODO : LTR. Need Stream Specific ConsumerArn to be passed from Customer private String getOrCreateConsumerArn(String streamName) { if (consumerArn != null) { return consumerArn; 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 0b7df88d..3af3dcf5 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 @@ -327,7 +327,7 @@ public class KinesisDataFetcher implements DataFetcher { } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } catch (InterruptedException e) { - // TODO: Check behaviorF + // TODO: Check behavior log.debug("{} : Interrupt called on method, shutdown initiated", streamAndShardId); throw new RuntimeException(e); } catch (TimeoutException e) { 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 96cec37a..740cbbef 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 @@ -36,7 +36,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.mockito.internal.verification.VerificationModeFactory.atMost; -import java.time.Instant; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -48,7 +48,6 @@ import java.util.Set; import java.util.concurrent.RejectedExecutionException; import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.Stream; import com.google.common.base.Joiner; import com.google.common.collect.Sets; @@ -64,8 +63,6 @@ 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; -import software.amazon.awssdk.utils.Either; -import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.checkpoint.Checkpoint; import software.amazon.kinesis.checkpoint.CheckpointConfig; import software.amazon.kinesis.checkpoint.CheckpointFactory; @@ -184,6 +181,7 @@ public class SchedulerTest { }}; when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList); + when(multiStreamTracker.waitPeriodToDeleteOldStreams()).thenReturn(Duration.ofHours(1L)); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); when(shardSyncTaskManager.executeShardSyncTask()).thenReturn(new TaskResult(null)); @@ -492,7 +490,7 @@ public class SchedulerTest { scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); - when(scheduler.getOldStreamDeferredDeletionPeriodMillis()).thenReturn(0L); + when(multiStreamTracker.waitPeriodToDeleteOldStreams()).thenReturn(Duration.ZERO); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Set expectedSyncedStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( @@ -563,7 +561,7 @@ public class SchedulerTest { scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); - when(scheduler.getOldStreamDeferredDeletionPeriodMillis()).thenReturn(0L); + when(multiStreamTracker.waitPeriodToDeleteOldStreams()).thenReturn(Duration.ZERO); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Set expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( From 61e500d4fdf0d1e8695c4cb2804739c549d0e47a Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 21 Apr 2020 22:17:51 -0700 Subject: [PATCH 071/159] Adding logging for streanms to be deleted --- .../software/amazon/kinesis/coordinator/Scheduler.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 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 22be7fc7..12cbae8f 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 @@ -458,8 +458,7 @@ public class Scheduler implements Runnable { if (shouldSyncStreamsNow()) { final Map newStreamConfigMap = new HashMap<>(); - final long waitPeriodToDeleteOldStreamsMillis = multiStreamTracker.waitPeriodToDeleteOldStreams() - .toMillis(); + final Duration waitPeriodToDeleteOldStreams = multiStreamTracker.waitPeriodToDeleteOldStreams(); // Making an immutable copy newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); @@ -525,13 +524,17 @@ public class Scheduler implements Runnable { final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream() .filter(streamIdentifier -> Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() - >= waitPeriodToDeleteOldStreamsMillis).collect(Collectors.toSet()); + >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); streamsSynced.addAll(deleteMultiStreamLeases(staleStreamIdsToBeDeleted)); // Purge the active streams from stale streams list. final Set staleStreamIdsToBeRevived = staleStreamIdDeletionDecisionMap.get(true); removeActiveStreamsFromStaleStreamsList(staleStreamIdsToBeRevived); + log.warn("Streams enqueued for deletion for lease table cleanup along with their scheduled time for deletion: {} ", + staleStreamDeletionMap.entrySet().stream().collect(Collectors + .toMap(Map.Entry::getKey, entry -> entry.getValue().plus(waitPeriodToDeleteOldStreams)))); + streamSyncWatch.reset().start(); } return streamsSynced; From 167ecfb08cef4e3febb9de2422fd387e8ce59c5c Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 22 Apr 2020 00:15:36 -0700 Subject: [PATCH 072/159] Add licensing information --- .../amazon/kinesis/leases/MultiStreamLease.java | 15 +++++++++++++++ .../DynamoDBMultiStreamLeaseSerializer.java | 15 +++++++++++++++ 2 files changed, 30 insertions(+) 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 96818f74..c8811354 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 @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.leases; import lombok.EqualsAndHashCode; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java index d703a970..78c9c6c4 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBMultiStreamLeaseSerializer.java @@ -1,3 +1,18 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.leases.dynamodb; import lombok.NoArgsConstructor; From 038524e0b154a78b0991edb4046dc966739313b5 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 24 Apr 2020 00:01:19 -0700 Subject: [PATCH 073/159] Introducing lease deletion strategy for multistreaming --- .../amazon/kinesis/common/ConfigsBuilder.java | 10 +- .../amazon/kinesis/coordinator/Scheduler.java | 178 ++++++++++-------- .../amazon/kinesis/metrics/MetricsUtil.java | 5 + .../FormerStreamsLeasesDeletionStrategy.java | 109 +++++++++++ .../kinesis/processor/MultiStreamTracker.java | 12 +- .../kinesis/coordinator/SchedulerTest.java | 177 ++++++++++++++++- 6 files changed, 399 insertions(+), 92 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.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 9595fdf9..09d28495 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/ConfigsBuilder.java @@ -124,7 +124,10 @@ public class ConfigsBuilder { * @param workerIdentifier * @param shardRecordProcessorFactory */ - public ConfigsBuilder(@NonNull String streamName, @NonNull String applicationName, @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + public ConfigsBuilder(@NonNull String streamName, @NonNull String applicationName, + @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, + @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, + @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { this.appStreamTracker = Either.right(streamName); this.applicationName = applicationName; this.kinesisClient = kinesisClient; @@ -144,7 +147,10 @@ public class ConfigsBuilder { * @param workerIdentifier * @param shardRecordProcessorFactory */ - public ConfigsBuilder(@NonNull MultiStreamTracker multiStreamTracker, @NonNull String applicationName, @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { + public ConfigsBuilder(@NonNull MultiStreamTracker multiStreamTracker, @NonNull String applicationName, + @NonNull KinesisAsyncClient kinesisClient, @NonNull DynamoDbAsyncClient dynamoDBClient, + @NonNull CloudWatchAsyncClient cloudWatchClient, @NonNull String workerIdentifier, + @NonNull ShardRecordProcessorFactory shardRecordProcessorFactory) { this.appStreamTracker = Either.left(multiStreamTracker); this.applicationName = applicationName; this.kinesisClient = kinesisClient; 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 12cbae8f..e520fdce 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 @@ -26,7 +26,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -40,6 +39,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; import lombok.AccessLevel; @@ -83,7 +83,11 @@ import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.CloudWatchMetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; +import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.processor.Checkpointer; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; @@ -92,6 +96,8 @@ import software.amazon.kinesis.retrieval.AggregatorUtil; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RetrievalConfig; +import static software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy.StreamsLeasesDeletionType; + /** * */ @@ -106,6 +112,10 @@ public class Scheduler implements Runnable { private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; private static final long HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS = 5000L; private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; + 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(); @@ -143,6 +153,7 @@ public class Scheduler implements Runnable { private final boolean isMultiStreamMode; private final Map currentStreamConfigMap; private MultiStreamTracker multiStreamTracker; + private FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy; private final long listShardsBackoffTimeMillis; private final int maxListShardsRetryAttempts; private final LeaseRefresher leaseRefresher; @@ -212,6 +223,7 @@ public class Scheduler implements Runnable { this.currentStreamConfigMap = this.retrievalConfig.appStreamTracker().map( multiStreamTracker -> { this.multiStreamTracker = multiStreamTracker; + this.formerStreamsLeasesDeletionStrategy = multiStreamTracker.formerStreamsLeasesDeletionStrategy(); return multiStreamTracker.streamConfigList().stream() .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc)); }, @@ -457,92 +469,108 @@ public class Scheduler implements Runnable { final Set streamsSynced = new HashSet<>(); if (shouldSyncStreamsNow()) { - final Map newStreamConfigMap = new HashMap<>(); - final Duration waitPeriodToDeleteOldStreams = multiStreamTracker.waitPeriodToDeleteOldStreams(); - // Making an immutable copy - newStreamConfigMap.putAll(multiStreamTracker.streamConfigList().stream() - .collect(Collectors.toMap(sc -> sc.streamIdentifier(), sc -> sc))); + final MetricsScope metricsScope = MetricsUtil.createMetricsWithOperation(metricsFactory, MULTI_STREAM_TRACKER); - List leases; + try { - // This is done to ensure that we clean up the stale streams lingering in the lease table. - if (!leasesSyncedOnAppInit && isMultiStreamMode) { - leases = fetchMultiStreamLeases(); - syncStreamsFromLeaseTableOnAppInit(leases); - leasesSyncedOnAppInit = true; - } + 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))); - // For new streams discovered, do a shard sync and update the currentStreamConfigMap - for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { - if (!currentStreamConfigMap.containsKey(streamIdentifier)) { - log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); - ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.syncShardAndLeaseInfo(); - currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); - streamsSynced.add(streamIdentifier); - } else { - if (log.isDebugEnabled()) { - log.debug(streamIdentifier + " is already being processed - skipping shard sync."); + List leases; + + // This is done to ensure that we clean up the stale streams lingering in the lease table. + if (!leasesSyncedOnAppInit && isMultiStreamMode) { + leases = fetchMultiStreamLeases(); + syncStreamsFromLeaseTableOnAppInit(leases); + leasesSyncedOnAppInit = true; + } + + // For new streams discovered, do a shard sync and update the currentStreamConfigMap + for (StreamIdentifier streamIdentifier : newStreamConfigMap.keySet()) { + if (!currentStreamConfigMap.containsKey(streamIdentifier)) { + log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.syncShardAndLeaseInfo(); + currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); + streamsSynced.add(streamIdentifier); + } else { + if (log.isDebugEnabled()) { + log.debug(streamIdentifier + " is already being processed - skipping shard sync."); + } } } - } - // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion. - // It is assumed that all the workers will always have the latest and consistent snapshot of streams - // from the multiStreamTracker. - // - // The following streams transition state among two workers are NOT considered safe, where Worker 2, on - // initialization learn about D from lease table and delete the leases for D, as it is not available - // in its latest MultiStreamTracker. - // Worker 1 : A,B,C -> A,B,C,D (latest) - // Worker 2 : BOOTS_UP -> A,B,C (stale) - // - // The following streams transition state among two workers are NOT considered safe, where Worker 2 might - // end up deleting the leases for A and D and loose progress made so far. - // Worker 1 : A,B,C -> A,B,C,D (latest) - // Worker 2 : A,B,C -> B,C (stale/partial) - // - // In order to give workers with stale stream info, sufficient time to learn about the new streams - // before attempting to delete it, we will be deferring the leases deletion based on the - // defer time period. + final Consumer enqueueStreamLeaseDeletionOperation = streamIdentifier -> { + if (!newStreamConfigMap.containsKey(streamIdentifier)) { + staleStreamDeletionMap.putIfAbsent(streamIdentifier, Instant.now()); + } + }; - Iterator currentStreamConfigIter = currentStreamConfigMap.keySet().iterator(); - while (currentStreamConfigIter.hasNext()) { - StreamIdentifier streamIdentifier = currentStreamConfigIter.next(); - if (!newStreamConfigMap.containsKey(streamIdentifier)) { - staleStreamDeletionMap.putIfAbsent(streamIdentifier, Instant.now()); + if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) { + // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion. + // It is assumed that all the workers will always have the latest and consistent snapshot of streams + // from the multiStreamTracker. + // + // The following streams transition state among two workers are NOT considered safe, where Worker 2, on + // initialization learn about D from lease table and delete the leases for D, as it is not available + // in its latest MultiStreamTracker. + // Worker 1 : A,B,C -> A,B,C,D (latest) + // Worker 2 : BOOTS_UP -> A,B,C (stale) + // + // The following streams transition state among two workers are NOT considered safe, where Worker 2 might + // end up deleting the leases for A and D and loose progress made so far. + // Worker 1 : A,B,C -> A,B,C,D (latest) + // Worker 2 : A,B,C -> B,C (stale/partial) + // + // In order to give workers with stale stream info, sufficient time to learn about the new streams + // before attempting to delete it, we will be deferring the leases deletion based on the + // defer time period. + + currentStreamConfigMap.keySet().stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier)); + + } else if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION) { + Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiers()).ifPresent( + streamIdentifiers -> streamIdentifiers.stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier))); } + + // Now let's scan the streamIdentifiers eligible for deferred deletion and delete them. + // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and + // the streamIdentifiers are not present in the latest snapshot. + final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors + .partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), Collectors.toSet())); + final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier -> + Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); + final Set deletedStreamsLeases = deleteMultiStreamLeases(staleStreamIdsToBeDeleted); + streamsSynced.addAll(deletedStreamsLeases); + + // Purge the active streams from stale streams list. + final Set staleStreamIdsToBeRevived = staleStreamIdDeletionDecisionMap.get(true); + removeStreamsFromStaleStreamsList(staleStreamIdsToBeRevived); + + log.warn( + "Streams enqueued for deletion for lease table cleanup along with their scheduled time for deletion: {} ", + staleStreamDeletionMap.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, + entry -> entry.getValue().plus(waitPeriodToDeleteOldStreams)))); + + streamSyncWatch.reset().start(); + + MetricsUtil.addCount(metricsScope, ACTIVE_STREAMS_COUNT, newStreamConfigMap.size(), MetricsLevel.SUMMARY); + MetricsUtil.addCount(metricsScope, PENDING_STREAMS_DELETION_COUNT, staleStreamDeletionMap.size(), + MetricsLevel.SUMMARY); + MetricsUtil.addCount(metricsScope, DELETED_STREAMS_COUNT, deletedStreamsLeases.size(), MetricsLevel.SUMMARY); + } finally { + MetricsUtil.endScope(metricsScope); } - - // Now let's scan the streamIdentifiers eligible for deferred deletion and delete them. - // StreamIdentifiers are eligible for deletion only when the deferment period has elapsed and - // the streamIdentifiers are not present in the latest snapshot. - final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet() - .stream().collect(Collectors - .partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), - Collectors.toSet())); - final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream() - .filter(streamIdentifier -> - Duration.between(staleStreamDeletionMap.get(streamIdentifier), Instant.now()).toMillis() - >= waitPeriodToDeleteOldStreams.toMillis()).collect(Collectors.toSet()); - streamsSynced.addAll(deleteMultiStreamLeases(staleStreamIdsToBeDeleted)); - - // Purge the active streams from stale streams list. - final Set staleStreamIdsToBeRevived = staleStreamIdDeletionDecisionMap.get(true); - removeActiveStreamsFromStaleStreamsList(staleStreamIdsToBeRevived); - - log.warn("Streams enqueued for deletion for lease table cleanup along with their scheduled time for deletion: {} ", - staleStreamDeletionMap.entrySet().stream().collect(Collectors - .toMap(Map.Entry::getKey, entry -> entry.getValue().plus(waitPeriodToDeleteOldStreams)))); - - streamSyncWatch.reset().start(); } return streamsSynced; } - @VisibleForTesting - boolean shouldSyncStreamsNow() { - return isMultiStreamMode && (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); + @VisibleForTesting boolean shouldSyncStreamsNow() { + return isMultiStreamMode && + (streamSyncWatch.elapsed(TimeUnit.MILLISECONDS) > NEW_STREAM_CHECK_INTERVAL_MILLIS); } private void syncStreamsFromLeaseTableOnAppInit(List leases) { @@ -561,7 +589,7 @@ public class Scheduler implements Runnable { return (List) ((List) leaseCoordinator.leaseRefresher().listLeases()); } - private void removeActiveStreamsFromStaleStreamsList(Set streamIdentifiers) { + private void removeStreamsFromStaleStreamsList(Set streamIdentifiers) { for(StreamIdentifier streamIdentifier : streamIdentifiers) { staleStreamDeletionMap.remove(streamIdentifier); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java index a2f9d84b..20c7c244 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/metrics/MetricsUtil.java @@ -94,6 +94,11 @@ public class MetricsUtil { metricsScope.addData(metricName, success ? 1 : 0, StandardUnit.COUNT, metricsLevel); } + public static void addCount(@NonNull final MetricsScope metricsScope, final String dimension, + final long count, @NonNull final MetricsLevel metricsLevel) { + metricsScope.addData(dimension, count, StandardUnit.COUNT, metricsLevel); + } + public static void endScope(@NonNull final MetricsScope metricsScope) { metricsScope.end(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java new file mode 100644 index 00000000..5c202040 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java @@ -0,0 +1,109 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.processor; + +import software.amazon.kinesis.common.StreamIdentifier; + +import java.time.Duration; +import java.util.List; + +/** + * Strategy for cleaning up the leases for former streams. + */ +public interface FormerStreamsLeasesDeletionStrategy { + + /** + * StreamIdentifiers for which leases needs to be cleaned up in the lease table. + * @return + */ + List streamIdentifiers(); + + /** + * Duration to wait before deleting the leases for this stream. + * @return + */ + Duration waitPeriodToDeleteFormerStreams(); + + /** + * Strategy type for deleting the leases of former streams + * @return + */ + StreamsLeasesDeletionType leaseDeletionType(); + + /** + * StreamsLeasesDeletionType identifying the different lease cleanup strategies. + */ + enum StreamsLeasesDeletionType { + NO_STREAMS_LEASES_DELETION, + FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION, + PROVIDED_STREAMS_DEFERRED_DELETION + } + + /** + * Strategy for not cleaning up leases for former streams. + */ + final class NoLeaseDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final List streamIdentifiers() { + throw new UnsupportedOperationException("StreamIdentifiers not required"); + } + + @Override + public final Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION; + } + } + + /** + * Strategy for auto detection the old of former streams based on the {@link MultiStreamTracker#streamConfigList()} + * and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()} + */ + abstract class AutoDetectionAndDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final List streamIdentifiers() { + throw new UnsupportedOperationException("StreamIdentifiers not required"); + } + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION; + } + } + + /** + * Strategy to detect the streams for deletion through {@link #streamIdentifiers()} provided by customer at runtime + * and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()} + */ + abstract class ProvidedStreamsDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION; + } + } + +} + + + + diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/MultiStreamTracker.java index 785778db..1b742509 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 @@ -17,7 +17,6 @@ package software.amazon.kinesis.processor; import software.amazon.kinesis.common.StreamConfig; -import java.time.Duration; import java.util.List; /** @@ -29,15 +28,18 @@ public interface MultiStreamTracker { /** * Returns the list of stream config, to be processed by the current application. - * Note that this method will be called periodically called by the KCL to learn about the new and old streams. + * 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(); /** - * Duration to wait before deleting the old streams in the lease table. - * @return Wait time before deleting old streams + * Strategy to delete leases of old streams in the lease table. + * Note that the strategy CANNOT be changed during the application runtime. + * + * @return StreamsLeasesDeletionStrategy */ - Duration waitPeriodToDeleteOldStreams(); + FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy(); } 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 740cbbef..d4f17917 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,6 +35,7 @@ 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; @@ -94,6 +95,7 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.Checkpointer; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; @@ -181,7 +183,12 @@ public class SchedulerTest { }}; when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList); - when(multiStreamTracker.waitPeriodToDeleteOldStreams()).thenReturn(Duration.ofHours(1L)); + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()) + .thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); when(shardSyncTaskManager.executeShardSyncTask()).thenReturn(new TaskResult(null)); @@ -442,7 +449,56 @@ public class SchedulerTest { } @Test - public final void testMultiStreamStaleStreamsAreNotDeletedImmediately() + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyAutoDeletionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamStaleStreamsAreNotDeletedImmediately(true); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyNoDeletionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy()); + testMultiStreamStaleStreamsAreNotDeletedImmediately(false); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiers() { + return null; + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamStaleStreamsAreNotDeletedImmediately(false); + } + + @Test + public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy2() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiers() { + return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(ArrayList::new)); + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamStaleStreamsAreNotDeletedImmediately(true); + } + + private final void testMultiStreamStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -457,6 +513,7 @@ public class SchedulerTest { retrievalConfig = new RetrievalConfig(kinesisClient, multiStreamTracker, applicationName) .retrievalFactory(retrievalFactory); when(multiStreamTracker.streamConfigList()).thenReturn(streamConfigList1, streamConfigList2); + scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); @@ -467,12 +524,59 @@ public class SchedulerTest { Assert.assertEquals(Sets.newHashSet(), syncedStreams); Assert.assertEquals(Sets.newHashSet(streamConfigList1), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); - Assert.assertEquals(expectedPendingStreams, + Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams : Sets.newHashSet(), scheduler.staleStreamDeletionMap().keySet()); } @Test - public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod() + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithAutoDetectionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null); + } + + @Test + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiers() { + return null; + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + HashSet currentStreamConfigMapOverride = IntStream.range(1, 5).mapToObj( + streamId -> new StreamConfig(StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(HashSet::new)); + testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(false, currentStreamConfigMapOverride); + } + + @Test + public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy2() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiers() { + return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( + Collectors.toCollection(ArrayList::new)); + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); + testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(true, null); + } + + private final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriod(boolean expectSyncedStreams, Set currentStreamConfigMapOverride) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -490,20 +594,69 @@ public class SchedulerTest { scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); - when(multiStreamTracker.waitPeriodToDeleteOldStreams()).thenReturn(Duration.ZERO); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Set expectedSyncedStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( Collectors.toCollection(HashSet::new)); - Assert.assertEquals(expectedSyncedStreams, syncedStreams); - Assert.assertEquals(Sets.newHashSet(streamConfigList2), + Assert.assertEquals(expectSyncedStreams ? expectedSyncedStreams : Sets.newHashSet(), syncedStreams); + Assert.assertEquals(currentStreamConfigMapOverride == null ? Sets.newHashSet(streamConfigList2) : currentStreamConfigMapOverride, Sets.newHashSet(scheduler.currentStreamConfigMap().values())); Assert.assertEquals(Sets.newHashSet(), scheduler.staleStreamDeletionMap().keySet()); } @Test - public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately() + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithAutoDetectionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithNoDeletionStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy()); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiers() { + return null; + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false); + } + + @Test + public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy2() + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { + @Override public List streamIdentifiers() { + return IntStream.range(1, 3) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(ArrayList::new)); + } + + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofHours(1); + } + }); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true); + } + + private final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -538,7 +691,7 @@ public class SchedulerTest { .collect(Collectors.toCollection(LinkedList::new)); Assert.assertEquals(Sets.newHashSet(expectedCurrentStreamConfigs), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); - Assert.assertEquals(expectedPendingStreams, + Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams: Sets.newHashSet(), scheduler.staleStreamDeletionMap().keySet()); } @@ -561,7 +714,11 @@ public class SchedulerTest { scheduler = spy(new Scheduler(checkpointConfig, coordinatorConfig, leaseManagementConfig, lifecycleConfig, metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); - when(multiStreamTracker.waitPeriodToDeleteOldStreams()).thenReturn(Duration.ZERO); + when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new AutoDetectionAndDeferredDeletionStrategy() { + @Override public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + }); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); Set expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( From 4d058ecf518fa4fdb23b3ca990eca6e7348d8a08 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 28 Apr 2020 15:53:22 -0700 Subject: [PATCH 074/159] Persisting hashrange in lease table --- .../kinesis/common/HashKeyRangeForLease.java | 54 +++++++++++++++++++ .../leases/HierarchicalShardSyncer.java | 8 ++- .../software/amazon/kinesis/leases/Lease.java | 27 ++++++++-- .../dynamodb/DynamoDBLeaseSerializer.java | 15 ++++++ .../leases/HierarchicalShardSyncerTest.java | 3 +- .../amazon/kinesis/leases/LeaseBuilder.java | 5 +- .../kinesis/leases/ShardObjectHelper.java | 2 +- .../dynamodb/DynamoDBLeaseRenewerTest.java | 4 +- 8 files changed, 109 insertions(+), 9 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java new file mode 100644 index 00000000..1d4529ff --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/HashKeyRangeForLease.java @@ -0,0 +1,54 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import com.google.common.base.Joiner; +import lombok.Data; +import lombok.experimental.Accessors; +import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.services.kinesis.model.HashKeyRange; + +@Data +@Accessors(fluent = true) +public class HashKeyRangeForLease { + + private static final String DELIM = ":"; + + private final String startingHashKey; + private final String endingHashKey; + + public String serialize() { + return Joiner.on(DELIM).join(startingHashKey, endingHashKey); + } + + @Override + public String toString() { + return serialize(); + } + + public static HashKeyRangeForLease deserialize(String hashKeyRange) { + final String[] hashKeyTokens = hashKeyRange.split(DELIM); + Validate.isTrue(hashKeyTokens.length == 2, "HashKeyRange should have exactly two tokens"); + // Assuming that startingHashKey and endingHashRange are not same. + Validate.isTrue(!hashKeyTokens[0].equals(hashKeyTokens[1]), "StartingHashKey and EndingHashKey should not be same"); + Validate.isTrue(hashKeyTokens[0].compareTo(hashKeyTokens[1]) < 0, "StartingHashKey must be less than EndingHashKey"); + return new HashKeyRangeForLease(hashKeyTokens[0], hashKeyTokens[1]); + } + + public static HashKeyRangeForLease fromHashKeyRange(HashKeyRange hashKeyRange) { + return new HashKeyRangeForLease(hashKeyRange.startingHashKey(), hashKeyRange.endingHashKey()); + } +} 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 debb89bb..714e7f4e 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 @@ -45,6 +45,7 @@ import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.services.kinesis.model.ShardFilterType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; @@ -57,6 +58,8 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRange; + /** * Helper class to sync leases with shards of the Kinesis stream. * It will create new leases/activities when it discovers new Kinesis shards (bootstrap/resharding). @@ -757,6 +760,7 @@ public class HierarchicalShardSyncer { } newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); newLease.ownerSwitchesSinceCheckpoint(0L); + newLease.hashKeyRange(fromHashKeyRange(childShard.hashKeyRange())); return newLease; } @@ -772,6 +776,7 @@ public class HierarchicalShardSyncer { newLease.ownerSwitchesSinceCheckpoint(0L); newLease.streamIdentifier(streamIdentifier.serialize()); newLease.shardId(childShard.shardId()); + newLease.hashKeyRange(fromHashKeyRange(childShard.hashKeyRange())); return newLease; } @@ -794,7 +799,7 @@ public class HierarchicalShardSyncer { } newLease.parentShardIds(parentShardIds); newLease.ownerSwitchesSinceCheckpoint(0L); - + newLease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); return newLease; } @@ -812,6 +817,7 @@ public class HierarchicalShardSyncer { newLease.ownerSwitchesSinceCheckpoint(0L); newLease.streamIdentifier(streamIdentifier.serialize()); newLease.shardId(shard.shardId()); + newLease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); return newLease; } 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 2d0ce8c2..c21e2203 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 @@ -20,6 +20,7 @@ import lombok.NoArgsConstructor; import lombok.NonNull; import lombok.ToString; import lombok.experimental.Accessors; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.util.Collection; @@ -96,6 +97,7 @@ public class Lease { private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); private Set childShardIds = new HashSet<>(); + private HashKeyRangeForLease hashKeyRangeForLease; /** * Copy constructor, used by clone(). @@ -105,7 +107,8 @@ public class Lease { protected Lease(Lease lease) { this(lease.leaseKey(), lease.leaseOwner(), lease.leaseCounter(), lease.concurrencyToken(), lease.lastCounterIncrementNanos(), lease.checkpoint(), lease.pendingCheckpoint(), - lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.childShardIds(), lease.pendingCheckpointState()); + lease.ownerSwitchesSinceCheckpoint(), lease.parentShardIds(), lease.childShardIds(), + lease.pendingCheckpointState(), lease.hashKeyRangeForLease()); } @Deprecated @@ -114,14 +117,14 @@ public class Lease { final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds) { this(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, pendingCheckpoint, - ownerSwitchesSinceCheckpoint, parentShardIds, new HashSet<>(), null); + ownerSwitchesSinceCheckpoint, parentShardIds, new HashSet<>(), null, null); } public Lease(final String leaseKey, final String leaseOwner, final Long leaseCounter, final UUID concurrencyToken, final Long lastCounterIncrementNanos, final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint, final Long ownerSwitchesSinceCheckpoint, final Set parentShardIds, final Set childShardIds, - final byte[] pendingCheckpointState) { + final byte[] pendingCheckpointState, final HashKeyRangeForLease hashKeyRangeForLease) { this.leaseKey = leaseKey; this.leaseOwner = leaseOwner; this.leaseCounter = leaseCounter; @@ -136,6 +139,9 @@ public class Lease { if (childShardIds != null) { this.childShardIds.addAll(childShardIds); } + if (hashKeyRangeForLease != null) { + this.hashKeyRangeForLease = hashKeyRangeForLease; + } this.pendingCheckpointState = pendingCheckpointState; } @@ -158,7 +164,8 @@ public class Lease { pendingCheckpoint(lease.pendingCheckpoint); pendingCheckpointState(lease.pendingCheckpointState); parentShardIds(lease.parentShardIds); - childShardIds(lease.childShardIds()); + childShardIds(lease.childShardIds); + hashKeyRange(lease.hashKeyRangeForLease); } /** @@ -274,6 +281,18 @@ public class Lease { this.childShardIds.addAll(childShardIds); } + /** + * Set the hash range key for this shard. + * @param hashKeyRangeForLease + */ + public void hashKeyRange(final HashKeyRangeForLease hashKeyRangeForLease) { + if(this.hashKeyRangeForLease == null) { + this.hashKeyRangeForLease = hashKeyRangeForLease; + } else { + throw new IllegalArgumentException("hashKeyRange is immutable"); + } + } + /** * Sets leaseOwner. * 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 85632500..61035be8 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 @@ -32,6 +32,7 @@ import software.amazon.awssdk.services.dynamodb.model.KeyType; import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.leases.DynamoUtils; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseSerializer; @@ -54,6 +55,7 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; private static final String CHILD_SHARD_IDS_KEY = "childShardIds"; + private static final String HASH_KEY_RANGE = "hashKeyRange"; @Override public Map toDynamoRecord(final Lease lease) { @@ -85,6 +87,10 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(PENDING_CHECKPOINT_STATE_KEY, DynamoUtils.createAttributeValue(lease.checkpoint().subSequenceNumber())); } + if(lease.hashKeyRangeForLease() != null) { + result.put(HASH_KEY_RANGE, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serialize())); + } + return result; } @@ -119,6 +125,11 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { leaseToUpdate.pendingCheckpointState(DynamoUtils.safeGetByteArray(dynamoRecord, PENDING_CHECKPOINT_STATE_KEY)); + final String hashKeyRange; + if (!Strings.isNullOrEmpty(hashKeyRange = DynamoUtils.safeGetString(dynamoRecord, HASH_KEY_RANGE))) { + leaseToUpdate.hashKeyRange(HashKeyRangeForLease.deserialize(hashKeyRange)); + } + return leaseToUpdate; } @@ -246,6 +257,10 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); } + if(lease.hashKeyRangeForLease() != null) { + result.put(HASH_KEY_RANGE, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serialize()))); + } + return result; } 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 cbf7add7..06433c1c 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 @@ -57,6 +57,7 @@ import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.services.kinesis.model.ShardFilterType; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; @@ -969,7 +970,7 @@ public class HierarchicalShardSyncerTest { parentShardIds.add(shard.adjacentParentShardId()); } return new Lease(shard.shardId(), leaseOwner, 0L, UUID.randomUUID(), 0L, checkpoint, null, 0L, - parentShardIds, new HashSet<>(), null); + parentShardIds, new HashSet<>(), null, HashKeyRangeForLease.fromHashKeyRange(shard.hashKeyRange())); }).collect(Collectors.toList()); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java index cf06f586..8f825875 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseBuilder.java @@ -20,6 +20,7 @@ import java.util.UUID; import lombok.Setter; import lombok.experimental.Accessors; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; @Setter @@ -36,9 +37,11 @@ public class LeaseBuilder { private Set parentShardIds = new HashSet<>(); private Set childShardIds = new HashSet<>(); private byte[] pendingCheckpointState; + private HashKeyRangeForLease hashKeyRangeForLease; public Lease build() { return new Lease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, checkpoint, - pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, pendingCheckpointState); + pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, + pendingCheckpointState, hashKeyRangeForLease); } } \ No newline at end of file diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java index f07a38f4..ee2504d8 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ShardObjectHelper.java @@ -68,7 +68,7 @@ public class ShardObjectHelper { String parentShardId, String adjacentParentShardId, SequenceNumberRange sequenceNumberRange) { - return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, null); + return newShard(shardId, parentShardId, adjacentParentShardId, sequenceNumberRange, HashKeyRange.builder().startingHashKey("1").endingHashKey("100").build()); } /** Helper method to create a new shard object. diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java index 1daf85b8..364a91be 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java @@ -37,6 +37,7 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.exceptions.DependencyException; @@ -55,7 +56,8 @@ public class DynamoDBLeaseRenewerTest { private LeaseRefresher leaseRefresher; private static Lease newLease(String leaseKey) { - return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, new HashSet<>(), new HashSet<>(), null); + return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, + new HashSet<>(), new HashSet<>(), null, new HashKeyRangeForLease("1", "2")); } @Before From 07c9529c149bd2d598da73e647742435a9c230bc Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 28 Apr 2020 15:58:50 -0700 Subject: [PATCH 075/159] doc comments --- .../kinesis/common/HashKeyRangeForLease.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 1d4529ff..2d45b130 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,6 +23,9 @@ import software.amazon.awssdk.services.kinesis.model.HashKeyRange; @Data @Accessors(fluent = true) +/** + * Lease POJO to hold the starting hashkey range and ending hashkey range of kinesis shards. + */ public class HashKeyRangeForLease { private static final String DELIM = ":"; @@ -30,6 +33,10 @@ public class HashKeyRangeForLease { private final String startingHashKey; private final String endingHashKey; + /** + * Serialize the HashKeyRangeForLease for persisting in external storage + * @return Serialized string + */ public String serialize() { return Joiner.on(DELIM).join(startingHashKey, endingHashKey); } @@ -39,6 +46,11 @@ public class HashKeyRangeForLease { return serialize(); } + /** + * Deserialize from serialized hashKeyRange string from external storage. + * @param hashKeyRange + * @return HashKeyRangeForLease + */ public static HashKeyRangeForLease deserialize(String hashKeyRange) { final String[] hashKeyTokens = hashKeyRange.split(DELIM); Validate.isTrue(hashKeyTokens.length == 2, "HashKeyRange should have exactly two tokens"); @@ -48,6 +60,11 @@ public class HashKeyRangeForLease { return new HashKeyRangeForLease(hashKeyTokens[0], hashKeyTokens[1]); } + /** + * Construct HashKeyRangeForLease from Kinesis HashKeyRange + * @param hashKeyRange + * @return HashKeyRangeForLease + */ public static HashKeyRangeForLease fromHashKeyRange(HashKeyRange hashKeyRange) { return new HashKeyRangeForLease(hashKeyRange.startingHashKey(), hashKeyRange.endingHashKey()); } From f69e9cf3baff5a0a8f3ff76c4191659b6b7f3ef7 Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Tue, 28 Apr 2020 16:32:29 -0700 Subject: [PATCH 076/159] Add getResponse override for ShardDetector (#19) (#21) --- .../kinesis/leases/KinesisShardDetector.java | 8 ++++- .../amazon/kinesis/leases/ShardDetector.java | 33 +++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) 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 a29c5ce4..9eb23c85 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 @@ -197,7 +197,7 @@ public class KinesisShardDetector implements ShardDetector { try { try { - result = FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request.build()), kinesisRequestTimeout); + result = getListShardsResponse(request.build()); } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } catch (InterruptedException e) { @@ -248,4 +248,10 @@ public class KinesisShardDetector implements ShardDetector { log.debug("{}. Age doesn't exceed limit of {} seconds.", message, listShardsCacheAllowedAgeInSeconds); return false; } + + @Override + public ListShardsResponse getListShardsResponse(ListShardsRequest request) throws + ExecutionException, TimeoutException, InterruptedException { + return FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request), kinesisRequestTimeout); + } } 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 2967a9fb..9eb2d17b 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 @@ -16,6 +16,8 @@ package software.amazon.kinesis.leases; import java.util.List; +import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; +import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.kinesis.common.StreamIdentifier; @@ -24,13 +26,44 @@ import software.amazon.kinesis.common.StreamIdentifier; * */ public interface ShardDetector { + + /** + * Gets shard based on shardId. + * + * @param shardId + * @return Shard + */ Shard shard(String shardId); + /** + * List shards. + * + * @return Shards + */ List listShards(); + /** + * List shards with shard filter. + * + * @param ShardFilter + * @return Shards + */ List listShardsWithFilter(ShardFilter shardFilter); + /** + * Gets stream identifier. + * + * @return StreamIdentifier + */ default StreamIdentifier streamIdentifier() { throw new UnsupportedOperationException("StreamName not available"); } + + /** + * Gets a list shards response based on the request. + * + * @param request list shards request + * @return ListShardsResponse which contains list shards response + */ + ListShardsResponse getListShardsResponse(ListShardsRequest request) throws Exception; } From bdec3bd66d44d4e0517b8ed5562b444a3c71dd2e Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 29 Apr 2020 15:23:14 -0700 Subject: [PATCH 077/159] Fixing comparison bug and addressing review comments --- .../kinesis/common/HashKeyRangeForLease.java | 54 ++++++++++--------- .../software/amazon/kinesis/leases/Lease.java | 8 ++- .../dynamodb/DynamoDBLeaseSerializer.java | 16 +++--- .../leases/HierarchicalShardSyncerTest.java | 26 +++++++-- .../dynamodb/DynamoDBLeaseRenewerTest.java | 2 +- 5 files changed, 67 insertions(+), 39 deletions(-) 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 2d45b130..063451a0 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 @@ -15,57 +15,63 @@ package software.amazon.kinesis.common; -import com.google.common.base.Joiner; -import lombok.Data; +import lombok.NonNull; +import lombok.Value; import lombok.experimental.Accessors; import org.apache.commons.lang3.Validate; import software.amazon.awssdk.services.kinesis.model.HashKeyRange; -@Data -@Accessors(fluent = true) +import java.math.BigInteger; + +@Value @Accessors(fluent = true) /** * Lease POJO to hold the starting hashkey range and ending hashkey range of kinesis shards. */ public class HashKeyRangeForLease { - private static final String DELIM = ":"; - - private final String startingHashKey; - private final String endingHashKey; + private final BigInteger startingHashKey; + private final BigInteger endingHashKey; /** - * Serialize the HashKeyRangeForLease for persisting in external storage - * @return Serialized string + * Serialize the startingHashKey for persisting in external storage + * + * @return Serialized startingHashKey */ - public String serialize() { - return Joiner.on(DELIM).join(startingHashKey, endingHashKey); + public String serializedStartingHashKey() { + return startingHashKey.toString(); } - @Override - public String toString() { - return serialize(); + /** + * Serialize the endingHashKey for persisting in external storage + * + * @return Serialized endingHashKey + */ + public String serializedEndingHashKey() { + return endingHashKey.toString(); } /** * Deserialize from serialized hashKeyRange string from external storage. - * @param hashKeyRange + * + * @param startingHashKeyStr + * @param endingHashKeyStr * @return HashKeyRangeForLease */ - public static HashKeyRangeForLease deserialize(String hashKeyRange) { - final String[] hashKeyTokens = hashKeyRange.split(DELIM); - Validate.isTrue(hashKeyTokens.length == 2, "HashKeyRange should have exactly two tokens"); - // Assuming that startingHashKey and endingHashRange are not same. - Validate.isTrue(!hashKeyTokens[0].equals(hashKeyTokens[1]), "StartingHashKey and EndingHashKey should not be same"); - Validate.isTrue(hashKeyTokens[0].compareTo(hashKeyTokens[1]) < 0, "StartingHashKey must be less than EndingHashKey"); - return new HashKeyRangeForLease(hashKeyTokens[0], hashKeyTokens[1]); + public static HashKeyRangeForLease deserialize(@NonNull String startingHashKeyStr, @NonNull String endingHashKeyStr) { + final BigInteger startingHashKey = new BigInteger(startingHashKeyStr); + final BigInteger endingHashKey = new BigInteger(endingHashKeyStr); + Validate.isTrue(startingHashKey.compareTo(endingHashKey) < 0, + "StartingHashKey %s must be less than EndingHashKey %s ", startingHashKeyStr, endingHashKeyStr); + return new HashKeyRangeForLease(startingHashKey, endingHashKey); } /** * Construct HashKeyRangeForLease from Kinesis HashKeyRange + * * @param hashKeyRange * @return HashKeyRangeForLease */ public static HashKeyRangeForLease fromHashKeyRange(HashKeyRange hashKeyRange) { - return new HashKeyRangeForLease(hashKeyRange.startingHashKey(), hashKeyRange.endingHashKey()); + return deserialize(hashKeyRange.startingHashKey(), hashKeyRange.endingHashKey()); } } 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 c21e2203..8ffd3bda 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 @@ -139,9 +139,7 @@ public class Lease { if (childShardIds != null) { this.childShardIds.addAll(childShardIds); } - if (hashKeyRangeForLease != null) { - this.hashKeyRangeForLease = hashKeyRangeForLease; - } + this.hashKeyRangeForLease = hashKeyRangeForLease; this.pendingCheckpointState = pendingCheckpointState; } @@ -286,9 +284,9 @@ public class Lease { * @param hashKeyRangeForLease */ public void hashKeyRange(final HashKeyRangeForLease hashKeyRangeForLease) { - if(this.hashKeyRangeForLease == null) { + if (this.hashKeyRangeForLease == null) { this.hashKeyRangeForLease = hashKeyRangeForLease; - } else { + } else if (!this.hashKeyRangeForLease.equals(hashKeyRangeForLease)) { throw new IllegalArgumentException("hashKeyRange is immutable"); } } 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 61035be8..8f293881 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 @@ -55,7 +55,8 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { private static final String PENDING_CHECKPOINT_STATE_KEY = "pendingCheckpointState"; private static final String PARENT_SHARD_ID_KEY = "parentShardId"; private static final String CHILD_SHARD_IDS_KEY = "childShardIds"; - private static final String HASH_KEY_RANGE = "hashKeyRange"; + private static final String STARTING_HASH_KEY = "startingHashKey"; + private static final String ENDING_HASH_KEY = "endingHashKey"; @Override public Map toDynamoRecord(final Lease lease) { @@ -88,7 +89,8 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { } if(lease.hashKeyRangeForLease() != null) { - result.put(HASH_KEY_RANGE, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serialize())); + result.put(STARTING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey())); + result.put(ENDING_HASH_KEY, DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey())); } return result; @@ -125,9 +127,10 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { leaseToUpdate.pendingCheckpointState(DynamoUtils.safeGetByteArray(dynamoRecord, PENDING_CHECKPOINT_STATE_KEY)); - final String hashKeyRange; - if (!Strings.isNullOrEmpty(hashKeyRange = DynamoUtils.safeGetString(dynamoRecord, HASH_KEY_RANGE))) { - leaseToUpdate.hashKeyRange(HashKeyRangeForLease.deserialize(hashKeyRange)); + final String startingHashKey, endingHashKey; + if (!Strings.isNullOrEmpty(startingHashKey = DynamoUtils.safeGetString(dynamoRecord, STARTING_HASH_KEY)) + && !Strings.isNullOrEmpty(endingHashKey = DynamoUtils.safeGetString(dynamoRecord, ENDING_HASH_KEY))) { + leaseToUpdate.hashKeyRange(HashKeyRangeForLease.deserialize(startingHashKey, endingHashKey)); } return leaseToUpdate; @@ -258,7 +261,8 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { } if(lease.hashKeyRangeForLease() != null) { - result.put(HASH_KEY_RANGE, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serialize()))); + result.put(STARTING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey()))); + result.put(ENDING_HASH_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey()))); } return result; 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 06433c1c..acfa3c51 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 @@ -20,6 +20,7 @@ package software.amazon.kinesis.leases; import static org.hamcrest.CoreMatchers.equalTo; 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; @@ -40,11 +41,13 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.Validate; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -120,7 +123,6 @@ 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)); } @@ -154,6 +156,8 @@ public class HierarchicalShardSyncerTest { final List newLeases = HierarchicalShardSyncer.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)); @@ -177,6 +181,7 @@ public class HierarchicalShardSyncerTest { final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(emptyLeaseTableSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, new HashSet<>(), MULTI_STREAM_ARGS); + validateHashRangeinLease(newLeases); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); @@ -213,6 +218,7 @@ public class HierarchicalShardSyncerTest { final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); + validateHashRangeinLease(newLeases); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set expectedLeaseShardIds = new HashSet<>(Arrays.asList(shardId0, shardId1)); assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); @@ -248,12 +254,22 @@ public class HierarchicalShardSyncerTest { final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds, MULTI_STREAM_ARGS); final Set newLeaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); + validateHashRangeinLease(newLeases); final Set expectedLeaseShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList(shardId0, shardId1))); assertThat(newLeases.size(), equalTo(expectedLeaseShardIds.size())); assertThat(newLeaseKeys, equalTo(expectedLeaseShardIds)); } + private void validateHashRangeinLease(List leases) { + final Consumer leaseValidation = lease -> { + Validate.notNull(lease.hashKeyRangeForLease()); + Validate.isTrue(lease.hashKeyRangeForLease().startingHashKey() + .compareTo(lease.hashKeyRangeForLease().endingHashKey()) < 0); + }; + leases.forEach(lease -> leaseValidation.accept(lease)); + } + /** * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) */ @@ -295,6 +311,7 @@ public class HierarchicalShardSyncerTest { final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); + validateHashRangeinLease(requestLeases); assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); assertThat(requestLeaseKeys, equalTo(expectedShardIds)); assertThat(extendedSequenceNumbers.size(), equalTo(1)); @@ -329,6 +346,7 @@ public class HierarchicalShardSyncerTest { final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); + validateHashRangeinLease(requestLeases); assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); assertThat(requestLeaseKeys, equalTo(expectedShardIds)); assertThat(extendedSequenceNumbers.size(), equalTo(1)); @@ -376,6 +394,8 @@ public class HierarchicalShardSyncerTest { assertThat(requestLeaseKeys, equalTo(expectedShardIds)); assertThat(extendedSequenceNumbers.size(), equalTo(1)); + validateHashRangeinLease(requestLeases); + extendedSequenceNumbers.forEach(seq -> assertThat(seq, equalTo(ExtendedSequenceNumber.LATEST))); verify(shardDetector, never()).listShards(); @@ -412,7 +432,7 @@ public class HierarchicalShardSyncerTest { 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(); @@ -444,7 +464,7 @@ public class HierarchicalShardSyncerTest { final Set requestLeaseKeys = requestLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set extendedSequenceNumbers = requestLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - + validateHashRangeinLease(requestLeases); assertThat(requestLeases.size(), equalTo(expectedShardIds.size())); assertThat(extendedSequenceNumbers.size(), equalTo(0)); 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 364a91be..bfff4e92 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewerTest.java @@ -57,7 +57,7 @@ public class DynamoDBLeaseRenewerTest { private static Lease newLease(String leaseKey) { return new Lease(leaseKey, "LeaseOwner", 0L, UUID.randomUUID(), System.nanoTime(), null, null, null, - new HashSet<>(), new HashSet<>(), null, new HashKeyRangeForLease("1", "2")); + new HashSet<>(), new HashSet<>(), null, HashKeyRangeForLease.deserialize("1", "2")); } @Before From 4a323b8f1cd717b254ab102e4b1d7e21842e30c2 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 29 Apr 2020 15:56:15 -0700 Subject: [PATCH 078/159] Fixing a metrics bug --- .../src/main/java/software/amazon/kinesis/leases/Lease.java | 3 +-- .../kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java | 6 ++++-- 2 files changed, 5 insertions(+), 4 deletions(-) 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 8ffd3bda..427b3509 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 @@ -163,7 +163,6 @@ public class Lease { pendingCheckpointState(lease.pendingCheckpointState); parentShardIds(lease.parentShardIds); childShardIds(lease.childShardIds); - hashKeyRange(lease.hashKeyRangeForLease); } /** @@ -283,7 +282,7 @@ public class Lease { * Set the hash range key for this shard. * @param hashKeyRangeForLease */ - public void hashKeyRange(final HashKeyRangeForLease hashKeyRangeForLease) { + public void hashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) { if (this.hashKeyRangeForLease == null) { this.hashKeyRangeForLease = hashKeyRangeForLease; } else if (!this.hashKeyRangeForLease.equals(hashKeyRangeForLease)) { 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 a1e0afcc..544de8cd 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 @@ -271,7 +271,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { * {@inheritDoc} */ @Override - public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String shardId) + public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { verifyNotNull(lease, "lease cannot be null"); verifyNotNull(lease.leaseKey(), "leaseKey cannot be null"); @@ -302,8 +302,10 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { if(lease instanceof MultiStreamLease) { MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())); + MetricsUtil.addShardId(scope, ((MultiStreamLease) lease).shardId()); + } else { + MetricsUtil.addShardId(scope, shardId); } - MetricsUtil.addShardId(scope, shardId); } long startTime = System.currentTimeMillis(); From 4ce0591e79cca64c3745642eab56165ab2604611 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 29 Apr 2020 15:59:53 -0700 Subject: [PATCH 079/159] Adding hashkeyRange back --- .../src/main/java/software/amazon/kinesis/leases/Lease.java | 1 + 1 file changed, 1 insertion(+) 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 427b3509..3df5097e 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 @@ -163,6 +163,7 @@ public class Lease { pendingCheckpointState(lease.pendingCheckpointState); parentShardIds(lease.parentShardIds); childShardIds(lease.childShardIds); + hashKeyRange(lease.hashKeyRangeForLease); } /** From a73701ff870caa0464888bd749e5b963e7573247 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 30 Apr 2020 17:20:01 -0700 Subject: [PATCH 080/159] Lease I/F changes and drop lease fix --- .../kinesis/leases/LeaseCoordinator.java | 5 +++- .../amazon/kinesis/leases/LeaseRefresher.java | 8 ++--- .../amazon/kinesis/leases/LeaseRenewer.java | 5 +++- .../dynamodb/DynamoDBLeaseCoordinator.java | 4 +-- .../dynamodb/DynamoDBLeaseRefresher.java | 4 +-- .../leases/dynamodb/DynamoDBLeaseRenewer.java | 8 +++-- .../kinesis/lifecycle/ShutdownTask.java | 29 +++++++++---------- .../ExceptionThrowingLeaseRefresher.java | 6 ++-- 8 files changed, 37 insertions(+), 32 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java index 880fab4c..6437f339 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCoordinator.java @@ -92,6 +92,9 @@ public interface LeaseCoordinator { * * @param lease lease object containing updated values * @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease + * @param operation that performs updateLease + * @param singleStreamShardId for metrics emission in single stream mode. MultiStream mode will get the + * shardId from the lease object * * @return true if update succeeded, false otherwise * @@ -99,7 +102,7 @@ public interface LeaseCoordinator { * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws DependencyException if DynamoDB update fails in an unexpected way */ - boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String shardId) + boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException; /** diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java index f45c4cc2..fc3aba8b 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 @@ -99,7 +99,7 @@ public interface LeaseRefresher { throws DependencyException, InvalidStateException, ProvisionedThroughputException; /** - * @param shardId Get the lease for this shardId + * @param leaseKey Get the lease for this leasekey * * @throws InvalidStateException if lease table does not exist * @throws ProvisionedThroughputException if DynamoDB get fails due to lack of capacity @@ -107,7 +107,7 @@ public interface LeaseRefresher { * * @return lease for the specified shardId, or null if one doesn't exist */ - Lease getLease(String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException; + Lease getLease(String leaseKey) throws DependencyException, InvalidStateException, ProvisionedThroughputException; /** * Renew a lease by incrementing the lease counter. Conditional on the leaseCounter in DynamoDB matching the leaseCounter @@ -206,13 +206,13 @@ public interface LeaseRefresher { * Gets the current checkpoint of the shard. This is useful in the resharding use case * where we will wait for the parent shard to complete before starting on the records from a child shard. * - * @param shardId Checkpoint of this shard will be returned + * @param leaseKey Checkpoint of this shard will be returned * @return Checkpoint of this shard, or null if the shard record doesn't exist. * * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws InvalidStateException if lease table does not exist * @throws DependencyException if DynamoDB update fails in an unexpected way */ - ExtendedSequenceNumber getCheckpoint(String shardId) + ExtendedSequenceNumber getCheckpoint(String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java index 9ed5616f..25ec5b45 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRenewer.java @@ -86,6 +86,9 @@ public interface LeaseRenewer { * * @param lease lease object containing updated data * @param concurrencyToken obtained by calling Lease.concurrencyToken for a currently held lease + * @param operation that performs updateLease + * @param singleStreamShardId shardId for metrics emission in single stream mode. MultiStream mode will get the + * shardId from the lease object * * @return true if update succeeds, false otherwise * @@ -93,7 +96,7 @@ public interface LeaseRenewer { * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws DependencyException if DynamoDB update fails in an unexpected way */ - boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String shardId) + boolean updateLease(Lease lease, UUID concurrencyToken, String operation, String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java index 3b7057de..78673f66 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseCoordinator.java @@ -349,8 +349,8 @@ public class DynamoDBLeaseCoordinator implements LeaseCoordinator { @Override public boolean updateLease(final Lease lease, final UUID concurrencyToken, final String operation, - final String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return leaseRenewer.updateLease(lease, concurrencyToken, operation, shardId); + final String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + return leaseRenewer.updateLease(lease, concurrencyToken, operation, singleStreamShardId); } /** 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 c5bb1f66..67e5abbe 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 @@ -663,10 +663,10 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { * {@inheritDoc} */ @Override - public ExtendedSequenceNumber getCheckpoint(String shardId) + public ExtendedSequenceNumber getCheckpoint(String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException { ExtendedSequenceNumber checkpoint = null; - Lease lease = getLease(shardId); + Lease lease = getLease(leaseKey); if (lease != null) { checkpoint = lease.checkpoint(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java index a1e0afcc..ecb0fc26 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 @@ -271,7 +271,7 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { * {@inheritDoc} */ @Override - public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String shardId) + public boolean updateLease(Lease lease, UUID concurrencyToken, @NonNull String operation, String singleStreamShardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { verifyNotNull(lease, "lease cannot be null"); verifyNotNull(lease.leaseKey(), "leaseKey cannot be null"); @@ -298,12 +298,14 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { } final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation); - if (StringUtils.isNotEmpty(shardId)) { + if (StringUtils.isNotEmpty(singleStreamShardId)) { if(lease instanceof MultiStreamLease) { MetricsUtil.addStreamId(scope, StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())); + MetricsUtil.addShardId(scope, ((MultiStreamLease) lease).shardId()); + } else { + MetricsUtil.addShardId(scope, singleStreamShardId); } - MetricsUtil.addShardId(scope, shardId); } long startTime = System.currentTimeMillis(); 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 bf07f6e2..9111b946 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 @@ -47,11 +47,8 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.UUID; -import java.util.function.Function; import java.util.stream.Collectors; /** @@ -92,7 +89,7 @@ public class ShutdownTask implements ConsumerTask { private final List childShards; - private static final Function shardInfoIdProvider = shardInfo -> shardInfo + private static final Function leaseKeyProvider = shardInfo -> shardInfo .streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()).orElse(shardInfo.shardId()); /* * Invokes ShardRecordProcessor shutdown() API. @@ -110,7 +107,7 @@ public class ShutdownTask implements ConsumerTask { try { try { log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", - shardInfoIdProvider.apply(shardInfo), shardInfo.concurrencyToken(), reason); + leaseKeyProvider.apply(shardInfo), shardInfo.concurrencyToken(), reason); final long startTime = System.currentTimeMillis(); if (reason == ShutdownReason.SHARD_END) { @@ -123,7 +120,7 @@ public class ShutdownTask implements ConsumerTask { createLeasesForChildShardsIfNotExist(); updateLeasesForChildShards(); } else { - log.warn("Shard {} no longer exists. Shutting down consumer with SHARD_END reason without creating leases for child shards.", shardInfoIdProvider.apply(shardInfo)); + log.warn("Shard {} no longer exists. Shutting down consumer with SHARD_END reason without creating leases for child shards.", leaseKeyProvider.apply(shardInfo)); } recordProcessorCheckpointer @@ -136,23 +133,23 @@ public class ShutdownTask implements ConsumerTask { throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); } - log.debug("Shutting down retrieval strategy for shard {}.", shardInfoIdProvider.apply(shardInfo)); + log.debug("Shutting down retrieval strategy for shard {}.", leaseKeyProvider.apply(shardInfo)); recordsPublisher.shutdown(); - log.debug("Record processor completed shutdown() for shard {}", shardInfoIdProvider.apply(shardInfo)); + log.debug("Record processor completed shutdown() for shard {}", leaseKeyProvider.apply(shardInfo)); return new TaskResult(null); } catch (Exception e) { if (e instanceof CustomerApplicationException) { - log.error("Shard {}: Application exception. ", shardInfoIdProvider.apply(shardInfo), e); + log.error("Shard {}: Application exception. ", leaseKeyProvider.apply(shardInfo), e); } else { - log.error("Shard {}: Caught exception: ", shardInfoIdProvider.apply(shardInfo), e); + log.error("Shard {}: Caught exception: ", leaseKeyProvider.apply(shardInfo), e); } exception = e; // backoff if we encounter an exception. try { Thread.sleep(this.backoffTimeMillis); } catch (InterruptedException ie) { - log.debug("Shard {}: Interrupted sleep", shardInfoIdProvider.apply(shardInfo), ie); + log.debug("Shard {}: Interrupted sleep", leaseKeyProvider.apply(shardInfo), ie); } } } finally { @@ -168,7 +165,7 @@ public class ShutdownTask implements ConsumerTask { if (lastCheckpointValue == null || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfoIdProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + + + leaseKeyProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + "See ShardRecordProcessor.shardEnded javadocs for more information."); } } @@ -177,7 +174,7 @@ public class ShutdownTask implements ConsumerTask { try { action.run(); } catch (Exception e) { - throw new CustomerApplicationException("Customer application throws exception for shard " + shardInfoIdProvider.apply(shardInfo) +": ", e); + throw new CustomerApplicationException("Customer application throws exception for shard " + leaseKeyProvider.apply(shardInfo) +": ", e); } finally { MetricsUtil.addLatency(metricsScope, RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); } @@ -195,12 +192,12 @@ public class ShutdownTask implements ConsumerTask { private void updateLeasesForChildShards() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - final Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfoIdProvider.apply(shardInfo)); + final Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); Set childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet()); final Lease updatedLease = currentLease.copy(); updatedLease.childShardIds(childShardIds); - leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, shardInfoIdProvider.apply(shardInfo)); + leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, leaseKeyProvider.apply(shardInfo)); } /* @@ -233,7 +230,7 @@ public class ShutdownTask implements ConsumerTask { } private void dropLease() { - Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId()); + Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); leaseCoordinator.dropLease(currentLease); if(currentLease != null) { log.warn("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); 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 d9d7d01e..81a49839 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java @@ -195,11 +195,11 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { } @Override - public Lease getLease(String shardId) + public Lease getLease(String leaseKey) throws DependencyException, InvalidStateException, ProvisionedThroughputException { throwExceptions("getLease", ExceptionThrowingLeaseRefresherMethods.GETLEASE); - return leaseRefresher.getLease(shardId); + return leaseRefresher.getLease(leaseKey); } @Override @@ -216,7 +216,7 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { } @Override - public ExtendedSequenceNumber getCheckpoint(final String shardId) + public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException { return null; } From 1036006bb4c06b10355e9786bca7b055cafd1661 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 5 May 2020 15:44:45 -0700 Subject: [PATCH 081/159] leaseKey parameter fix and logging change for shutdownTask --- .../amazon/kinesis/lifecycle/BlockOnParentShardTask.java | 5 ++++- .../software/amazon/kinesis/lifecycle/ShutdownTask.java | 7 ++++++- .../amazon/kinesis/lifecycle/ShutdownTaskTest.java | 5 +++++ 3 files changed, 15 insertions(+), 2 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java index 37a092e8..0894be69 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java @@ -25,6 +25,8 @@ import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.util.function.Function; + /** * Task to block until processing of all data records in the parent shard(s) is completed. * We check if we have checkpoint(s) for the parent shard(s). @@ -59,7 +61,8 @@ public class BlockOnParentShardTask implements ConsumerTask { try { boolean blockedOnParentShard = false; for (String shardId : shardInfo.parentShardIds()) { - Lease lease = leaseRefresher.getLease(shardId); + final String leaseKey = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardId).orElse(shardId); + final Lease lease = leaseRefresher.getLease(leaseKey); if (lease != null) { ExtendedSequenceNumber checkpoint = lease.checkpoint(); if ((checkpoint == null) || (!checkpoint.equals(ExtendedSequenceNumber.SHARD_END))) { 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 9111b946..2f05ece0 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 @@ -183,9 +183,13 @@ public class ShutdownTask implements ConsumerTask { private void createLeasesForChildShardsIfNotExist() throws DependencyException, InvalidStateException, ProvisionedThroughputException { for(ChildShard childShard : childShards) { - if(leaseCoordinator.getCurrentlyHeldLease(childShard.shardId()) == null) { + final String leaseKey = shardInfo.streamIdentifierSerOpt() + .map(s -> s + ":" + childShard.shardId()) + .orElse(childShard.shardId()); + if(leaseCoordinator.getCurrentlyHeldLease(leaseKey) == null) { final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); + log.info("Shard {}: Created child shard lease: {}", shardInfo.shardId(), leaseToCreate.leaseKey()); } } } @@ -198,6 +202,7 @@ public class ShutdownTask implements ConsumerTask { final Lease updatedLease = currentLease.copy(); updatedLease.childShardIds(childShardIds); leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, leaseKeyProvider.apply(shardInfo)); + log.info("Shard {}: Updated current lease {} with child shard information: {}", shardInfo.shardId(), currentLease.leaseKey(), childShardIds); } /* 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 c94a3266..7992d604 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 @@ -46,6 +46,7 @@ import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; @@ -107,6 +108,10 @@ public class ShutdownTaskTest { public void setUp() throws Exception { doNothing().when(recordsPublisher).shutdown(); when(recordProcessorCheckpointer.checkpointer()).thenReturn(checkpointer); + final Lease childLease = new Lease(); + childLease.leaseKey("childShardLeaseKey"); + when(hierarchicalShardSyncer.createLeaseForChildShard(Matchers.any(ChildShard.class), Matchers.any(StreamIdentifier.class))) + .thenReturn(childLease); shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); From 760c52d7bcaa1131cc1f7d0428e3cdaf26720524 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 5 May 2020 17:31:08 -0700 Subject: [PATCH 082/159] getLeaseKey --- .../java/software/amazon/kinesis/leases/ShardInfo.java | 8 ++++++-- .../amazon/kinesis/lifecycle/BlockOnParentShardTask.java | 2 +- .../software/amazon/kinesis/lifecycle/ShutdownTask.java | 4 +--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index c4b2968c..e0da7dec 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -149,9 +149,13 @@ public class ShardInfo { * @return lease key */ public static String getLeaseKey(ShardInfo shardInfo) { + return getLeaseKey(shardInfo, shardInfo.shardId()); + } + + public static String getLeaseKey(ShardInfo shardInfo, String shardId) { return shardInfo.streamIdentifierSerOpt().isPresent() ? - MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardInfo.shardId()) : - shardInfo.shardId(); + MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardId) : + shardId; } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java index 0894be69..8797085a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java @@ -61,7 +61,7 @@ public class BlockOnParentShardTask implements ConsumerTask { try { boolean blockedOnParentShard = false; for (String shardId : shardInfo.parentShardIds()) { - final String leaseKey = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardId).orElse(shardId); + final String leaseKey = ShardInfo.getLeaseKey(shardInfo, shardId); final Lease lease = leaseRefresher.getLease(leaseKey); if (lease != null) { ExtendedSequenceNumber checkpoint = lease.checkpoint(); 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 2f05ece0..9d53e75c 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 @@ -183,9 +183,7 @@ public class ShutdownTask implements ConsumerTask { private void createLeasesForChildShardsIfNotExist() throws DependencyException, InvalidStateException, ProvisionedThroughputException { for(ChildShard childShard : childShards) { - final String leaseKey = shardInfo.streamIdentifierSerOpt() - .map(s -> s + ":" + childShard.shardId()) - .orElse(childShard.shardId()); + final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId()); if(leaseCoordinator.getCurrentlyHeldLease(leaseKey) == null) { final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); From e86b1d1f0139efa5612c7ff54945da70096b64fd Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 5 May 2020 17:34:37 -0700 Subject: [PATCH 083/159] Adding shouldCleanupLeasesForDeletedStreams option to FormerStreamsLeasesDeletionStrategy and introducing OnlyDeletedStreamsLeasesCleanupStrategy. --- .../amazon/kinesis/coordinator/Scheduler.java | 22 ++++++++- .../FormerStreamsLeasesDeletionStrategy.java | 49 ++++++++++++++++++- .../kinesis/coordinator/SchedulerTest.java | 1 - 3 files changed, 69 insertions(+), 3 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 e520fdce..a20f86d1 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 @@ -26,6 +26,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -509,6 +510,25 @@ public class Scheduler implements Runnable { } }; + if (formerStreamsLeasesDeletionStrategy.shouldCleanupLeasesForDeletedStreams()) { + // We do lease sync for old streams, before leaving to the deletion strategy to delete leases for + // strategy detected leases. Also, for deleted streams we expect the shard sync to remove the + // leases. + Iterator currentSetOfStreamsIter = currentStreamConfigMap.keySet().iterator(); + while (currentSetOfStreamsIter.hasNext()) { + StreamIdentifier streamIdentifier = currentSetOfStreamsIter.next(); + if (!newStreamConfigMap.containsKey(streamIdentifier)) { + log.info("Found old/deleted stream: " + streamIdentifier + + ". Syncing shards of that stream."); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager( + currentStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.syncShardAndLeaseInfo(); + currentSetOfStreamsIter.remove(); + streamsSynced.add(streamIdentifier); + } + } + } + if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) { // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion. // It is assumed that all the workers will always have the latest and consistent snapshot of streams @@ -611,7 +631,7 @@ public class Scheduler implements Runnable { .groupingBy(MultiStreamLease::streamIdentifier, Collectors.toCollection(ArrayList::new))); } - log.warn("Found old/deleted stream: " + streamIdentifier + ". Deleting leases of this stream."); + log.warn("Found old/deleted stream: " + streamIdentifier + ". Directly deleting leases of this stream."); // Deleting leases will cause the workers to shutdown the record processors for these shards. if (deleteMultiStreamLeases(streamIdToShardsMap.get(streamIdentifier.serialize()))) { currentStreamConfigMap.remove(streamIdentifier); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java index 5c202040..e59266a4 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java @@ -38,11 +38,17 @@ public interface FormerStreamsLeasesDeletionStrategy { Duration waitPeriodToDeleteFormerStreams(); /** - * Strategy type for deleting the leases of former streams + * Strategy type for deleting the leases of former active streams. * @return */ StreamsLeasesDeletionType leaseDeletionType(); + /** + * Should the leases be cleaned up for deleted streams + * @return true if leases be cleaned up for deleted streams; false otherwise. + */ + boolean shouldCleanupLeasesForDeletedStreams(); + /** * StreamsLeasesDeletionType identifying the different lease cleanup strategies. */ @@ -71,6 +77,37 @@ public interface FormerStreamsLeasesDeletionStrategy { public final StreamsLeasesDeletionType leaseDeletionType() { return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION; } + + @Override + public final boolean shouldCleanupLeasesForDeletedStreams() { + return false; + } + } + + /** + * Strategy for not cleaning up leases for former streams. + */ + final class OnlyDeletedStreamsLeasesCleanupStrategy implements FormerStreamsLeasesDeletionStrategy { + + @Override + public final List streamIdentifiers() { + throw new UnsupportedOperationException("StreamIdentifiers not required"); + } + + @Override + public final Duration waitPeriodToDeleteFormerStreams() { + return Duration.ZERO; + } + + @Override + public final StreamsLeasesDeletionType leaseDeletionType() { + return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION; + } + + @Override + public final boolean shouldCleanupLeasesForDeletedStreams() { + return true; + } } /** @@ -88,6 +125,11 @@ public interface FormerStreamsLeasesDeletionStrategy { public final StreamsLeasesDeletionType leaseDeletionType() { return StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION; } + + @Override + public boolean shouldCleanupLeasesForDeletedStreams() { + return false; + } } /** @@ -100,6 +142,11 @@ public interface FormerStreamsLeasesDeletionStrategy { public final StreamsLeasesDeletionType leaseDeletionType() { return StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION; } + + @Override + public boolean shouldCleanupLeasesForDeletedStreams() { + return false; + } } } 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 d4f17917..a5fd2add 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 @@ -95,7 +95,6 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsConfig; import software.amazon.kinesis.processor.Checkpointer; -import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; import software.amazon.kinesis.processor.MultiStreamTracker; import software.amazon.kinesis.processor.ProcessorConfig; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; From d28d26c6350d121e1919a692e8807324be2ee214 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 6 May 2020 10:41:12 -0700 Subject: [PATCH 084/159] Adding java docs --- .../software/amazon/kinesis/leases/ShardInfo.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java index e0da7dec..aff3f6f0 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardInfo.java @@ -144,7 +144,7 @@ public class ShardInfo { } /** - * Utility method to derive lease key from ShardInfo + * Utility method to derive lease key from ShardInfo. * @param shardInfo * @return lease key */ @@ -152,10 +152,16 @@ public class ShardInfo { return getLeaseKey(shardInfo, shardInfo.shardId()); } - public static String getLeaseKey(ShardInfo shardInfo, String shardId) { + /** + * Utility method to derive lease key from ShardInfo and shardId to override. + * @param shardInfo + * @param shardIdOverride + * @return lease key + */ + public static String getLeaseKey(ShardInfo shardInfo, String shardIdOverride) { return shardInfo.streamIdentifierSerOpt().isPresent() ? - MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardId) : - shardId; + MultiStreamLease.getLeaseKey(shardInfo.streamIdentifierSerOpt().get(), shardIdOverride) : + shardIdOverride; } } From eb7a60ec624eb0e26a281e7d0fdadf2befb97c0a Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 29 Apr 2020 15:56:15 -0700 Subject: [PATCH 085/159] Fixing a metrics bug --- .../src/main/java/software/amazon/kinesis/leases/Lease.java | 1 - 1 file changed, 1 deletion(-) 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 3df5097e..427b3509 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 @@ -163,7 +163,6 @@ public class Lease { pendingCheckpointState(lease.pendingCheckpointState); parentShardIds(lease.parentShardIds); childShardIds(lease.childShardIds); - hashKeyRange(lease.hashKeyRangeForLease); } /** From 5cd40e4718e7f7576e1c0ba64cf03aee37e44936 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 30 Apr 2020 14:03:39 -0700 Subject: [PATCH 086/159] Lease Recovery and Blockoing parent bug fix --- .../coordinator/PeriodicShardSyncManager.java | 75 ++++++++++++--- .../amazon/kinesis/coordinator/Scheduler.java | 20 +--- .../leases/HierarchicalShardSyncer.java | 5 +- .../amazon/kinesis/leases/LeaseRefresher.java | 2 +- .../lifecycle/BlockOnParentShardTask.java | 2 - .../retrieval/kpl/ExtendedSequenceNumber.java | 5 + .../lifecycle/BlockOnParentShardTaskTest.java | 92 ++++++++++++++++++- 7 files changed, 164 insertions(+), 37 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 8acccce9..8df18207 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 @@ -18,20 +18,32 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardSyncTaskManager; +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.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; +import java.util.stream.Collectors; /** * The top level orchestrator for coordinating the periodic shard sync related @@ -46,25 +58,33 @@ class PeriodicShardSyncManager { private final String workerId; private final LeaderDecider leaderDecider; + private final LeaseRefresher leaseRefresher; private final Map currentStreamConfigMap; private final Function shardSyncTaskManagerProvider; private final ScheduledExecutorService shardSyncThreadPool; + private final boolean isMultiStreamingMode; private boolean isRunning; - PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, Map currentStreamConfigMap, - Function shardSyncTaskManagerProvider) { - this(workerId, leaderDecider, currentStreamConfigMap, shardSyncTaskManagerProvider, Executors.newSingleThreadScheduledExecutor()); + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher, + Map currentStreamConfigMap, + Function shardSyncTaskManagerProvider, boolean isMultiStreamingMode) { + this(workerId, leaderDecider, leaseRefresher, currentStreamConfigMap, shardSyncTaskManagerProvider, + Executors.newSingleThreadScheduledExecutor(), isMultiStreamingMode); } - PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, Map currentStreamConfigMap, - Function shardSyncTaskManagerProvider, ScheduledExecutorService shardSyncThreadPool) { + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher, + Map currentStreamConfigMap, + Function shardSyncTaskManagerProvider, + ScheduledExecutorService shardSyncThreadPool, boolean isMultiStreamingMode) { Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); this.workerId = workerId; this.leaderDecider = leaderDecider; + this.leaseRefresher = leaseRefresher; this.currentStreamConfigMap = currentStreamConfigMap; this.shardSyncTaskManagerProvider = shardSyncTaskManagerProvider; this.shardSyncThreadPool = shardSyncThreadPool; + this.isMultiStreamingMode = isMultiStreamingMode; } public synchronized TaskResult start() { @@ -116,24 +136,53 @@ class PeriodicShardSyncManager { private void runShardSync() { if (leaderDecider.isLeader(workerId)) { - for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { - final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfigEntry.getValue()); - if (!shardSyncTaskManager.syncShardAndLeaseInfo()) { - log.warn("Failed to submit shard sync task for stream {}. This could be due to the previous shard sync task not finished.", - shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + try { + final Map> streamToLeasesMap = getStreamToLeasesMap(currentStreamConfigMap.keySet()); + for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfigEntry.getValue()); + if (!shardSyncTaskManager.syncShardAndLeaseInfo()) { + log.warn( + "Failed to submit shard sync task for stream {}. This could be due to the previous shard sync task not finished.", + shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + } } + } catch (Exception e) { + // TODO : Log } } else { log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); } } + private Map> getStreamToLeasesMap(final Set streamIdentifiersToFilter) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final List leases = leaseRefresher.listLeases(); + if(!isMultiStreamingMode) { + Validate.isTrue(streamIdentifiersToFilter.size() == 1); + return Collections.singletonMap(streamIdentifiersToFilter.iterator().next(), leases); + } else { + final Map> streamToLeasesMap = new HashMap<>(); + for(Lease lease : leases) { + StreamIdentifier streamIdentifier = StreamIdentifier + .multiStreamInstance(((MultiStreamLease) lease).streamIdentifier()); + if(streamIdentifiersToFilter.contains(streamIdentifier)) { + streamToLeasesMap.computeIfAbsent(streamIdentifier, s -> new ArrayList<>()).add(lease); + } + } + return streamToLeasesMap; + } + } + /** * Checks if the entire hash range is covered * @return true if covered, false otherwise */ - public boolean hashRangeCovered() { - // TODO: Implement method - return true; + public boolean isHashRangeComplete(List leases) { + if(CollectionUtils.isNullOrEmpty(leases)) { + return false; + } else { +// leases.stream().filter(lease -> lease.checkpoint().isShardEnd()) + return false; + } } } 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 a20f86d1..d5eb1be6 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 @@ -111,7 +111,6 @@ public class Scheduler implements Runnable { private static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; - private static final long HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS = 5000L; private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; private static final String MULTI_STREAM_TRACKER = "MultiStreamTracker"; private static final String ACTIVE_STREAMS_COUNT = "ActiveStreams.Count"; @@ -289,8 +288,8 @@ public class Scheduler implements Runnable { this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(isMultiStreamMode); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager( - leaseManagementConfig.workerIdentifier(), leaderDecider, currentStreamConfigMap, - shardSyncTaskManagerProvider); + leaseManagementConfig.workerIdentifier(), leaderDecider, leaseRefresher, currentStreamConfigMap, + shardSyncTaskManagerProvider, isMultiStreamMode); } /** @@ -351,11 +350,10 @@ public class Scheduler implements Runnable { } else { log.info("LeaseCoordinator is already running. No need to start it."); } - log.info("Scheduling periodicShardSync)"); + log.info("Scheduling periodicShardSync"); // leaderElectedPeriodicShardSyncManager.start(shardSyncTasks); // TODO: enable periodicShardSync after https://github.com/jushkem/amazon-kinesis-client/pull/2 is merged // TODO: Determine if waitUntilHashRangeCovered() is needed. - //waitUntilHashRangeCovered(); streamSyncWatch.start(); isDone = true; } catch (LeasingException e) { @@ -398,18 +396,6 @@ public class Scheduler implements Runnable { return shouldInitiateLeaseSync; } - private void waitUntilHashRangeCovered() throws InterruptedException { - - // TODO: Currently this call is not in use. We may need to implement this method later. Created SIM to track the work: https://sim.amazon.com/issues/KinesisLTR-202 - // TODO: For future implementation, streamToShardSyncTaskManagerMap might not contain the most up to date snapshot of active streams. - // Should use currentStreamConfigMap to determine the streams to check. - while (!leaderElectedPeriodicShardSyncManager.hashRangeCovered()) { - // wait until entire hash range is covered - log.info("Hash range is not covered yet. Checking again in {} ms", HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS); - Thread.sleep(HASH_RANGE_COVERAGE_CHECK_FREQUENCY_MILLIS); - } - } - @VisibleForTesting void runProcessLoop() { try { 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 714e7f4e..2b2df48c 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 @@ -45,7 +45,6 @@ import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.services.kinesis.model.ShardFilterType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; -import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; @@ -341,7 +340,7 @@ public class HierarchicalShardSyncer { "Stream " + streamName + " is not in ACTIVE OR UPDATING state - will retry getting the shard list."); } - if (hashRangeOfShardsIsComplete(shards)) { + if (isHashRangeOfShardsComplete(shards)) { return shards; } @@ -359,7 +358,7 @@ public class HierarchicalShardSyncer { " is not in ACTIVE OR UPDATING state - will retry getting the shard list.")); } - private static boolean hashRangeOfShardsIsComplete(@NonNull List shards) { + private static boolean isHashRangeOfShardsComplete(@NonNull List shards) { if (shards.isEmpty()) { throw new IllegalStateException("No shards found when attempting to validate complete hash range."); 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 fc3aba8b..0d563a63 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 @@ -105,7 +105,7 @@ public interface LeaseRefresher { * @throws ProvisionedThroughputException if DynamoDB get fails due to lack of capacity * @throws DependencyException if DynamoDB get fails in an unexpected way * - * @return lease for the specified shardId, or null if one doesn't exist + * @return lease for the specified leaseKey, or null if one doesn't exist */ Lease getLease(String leaseKey) throws DependencyException, InvalidStateException, ProvisionedThroughputException; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java index 8797085a..fcb3ffde 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java @@ -25,8 +25,6 @@ import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.function.Function; - /** * Task to block until processing of all data records in the parent shard(s) is completed. * We check if we have checkpoint(s) for the parent shard(s). diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java index e2de9786..0c1c4a28 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/kpl/ExtendedSequenceNumber.java @@ -141,6 +141,11 @@ public class ExtendedSequenceNumber implements Comparable emptyParentShardIds = new ArrayList(); private ShardInfo shardInfo; @@ -107,6 +109,50 @@ public class BlockOnParentShardTaskTest { assertNull(result.getException()); } + /** + * Test call() when there are 1-2 parent shards that have been fully processed. + * @throws ProvisionedThroughputException + * @throws InvalidStateException + * @throws DependencyException + */ + @Test + public final void testCallWhenParentsHaveFinishedMultiStream() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + + ShardInfo shardInfo = null; + BlockOnParentShardTask task = null; + String parent1LeaseKey = streamId + ":" + "shardId-1"; + String parent2LeaseKey = streamId + ":" + "shardId-2"; + String parent1ShardId = "shardId-1"; + String parent2ShardId = "shardId-2"; + List parentShardIds = new ArrayList<>(); + TaskResult result = null; + + Lease parent1Lease = new Lease(); + parent1Lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + Lease parent2Lease = new Lease(); + parent2Lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + + LeaseRefresher leaseRefresher = mock(LeaseRefresher.class); + when(leaseRefresher.getLease(parent1LeaseKey)).thenReturn(parent1Lease); + when(leaseRefresher.getLease(parent2LeaseKey)).thenReturn(parent2Lease); + + // test single parent + parentShardIds.add(parent1ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, + streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNull(result.getException()); + + // test two parents + parentShardIds.add(parent2ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNull(result.getException()); + } + /** * Test call() when there are 1-2 parent shards that have NOT been fully processed. * @throws ProvisionedThroughputException @@ -115,7 +161,7 @@ public class BlockOnParentShardTaskTest { */ @Test public final void testCallWhenParentsHaveNotFinished() - throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throws DependencyException, InvalidStateException, ProvisionedThroughputException { ShardInfo shardInfo = null; BlockOnParentShardTask task = null; @@ -149,6 +195,50 @@ public class BlockOnParentShardTaskTest { assertNotNull(result.getException()); } + /** + * Test call() when there are 1-2 parent shards that have NOT been fully processed. + * @throws ProvisionedThroughputException + * @throws InvalidStateException + * @throws DependencyException + */ + @Test + public final void testCallWhenParentsHaveNotFinishedMultiStream() + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + + ShardInfo shardInfo = null; + BlockOnParentShardTask task = null; + String parent1LeaseKey = streamId + ":" + "shardId-1"; + String parent2LeaseKey = streamId + ":" + "shardId-2"; + String parent1ShardId = "shardId-1"; + String parent2ShardId = "shardId-2"; + List parentShardIds = new ArrayList<>(); + TaskResult result = null; + + Lease parent1Lease = new Lease(); + parent1Lease.checkpoint(ExtendedSequenceNumber.LATEST); + Lease parent2Lease = new Lease(); + // mock a sequence number checkpoint + parent2Lease.checkpoint(new ExtendedSequenceNumber("98182584034")); + + LeaseRefresher leaseRefresher = mock(LeaseRefresher.class); + when(leaseRefresher.getLease(parent1LeaseKey)).thenReturn(parent1Lease); + when(leaseRefresher.getLease(parent2LeaseKey)).thenReturn(parent2Lease); + + // test single parent + parentShardIds.add(parent1ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNotNull(result.getException()); + + // test two parents + parentShardIds.add(parent2ShardId); + shardInfo = new ShardInfo(shardId, concurrencyToken, parentShardIds, ExtendedSequenceNumber.TRIM_HORIZON, streamId); + task = new BlockOnParentShardTask(shardInfo, leaseRefresher, backoffTimeInMillis); + result = task.call(); + assertNotNull(result.getException()); + } + /** * Test call() with 1 parent shard before and after it is completely processed. * @throws ProvisionedThroughputException From eb00229602cbf1d988e13bd07be6ef51f5c7e733 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 1 May 2020 01:28:48 -0700 Subject: [PATCH 087/159] Hole detection logic --- .../coordinator/PeriodicShardSyncManager.java | 98 ++++++++++- .../software/amazon/kinesis/leases/Lease.java | 2 + .../PeriodicShardSyncManagerTest.java | 155 ++++++++++++++++++ 3 files changed, 248 insertions(+), 7 deletions(-) create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java 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 8df18207..3edec0ec 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 @@ -14,14 +14,16 @@ */ package software.amazon.kinesis.coordinator; +import com.google.common.annotations.VisibleForTesting; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; -import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.MultiStreamLease; @@ -29,19 +31,21 @@ import software.amazon.kinesis.leases.ShardSyncTaskManager; 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.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; +import java.io.Serializable; +import java.math.BigInteger; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; import java.util.stream.Collectors; @@ -55,6 +59,8 @@ import java.util.stream.Collectors; class PeriodicShardSyncManager { private static final long INITIAL_DELAY = 60 * 1000L; private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 5 * 60 * 1000L; + private static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; + private static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); private final String workerId; private final LeaderDecider leaderDecider; @@ -177,12 +183,90 @@ class PeriodicShardSyncManager { * Checks if the entire hash range is covered * @return true if covered, false otherwise */ - public boolean isHashRangeComplete(List leases) { - if(CollectionUtils.isNullOrEmpty(leases)) { + private boolean isHashRangeCompleteForLeases(List leases) { + if (CollectionUtils.isNullOrEmpty(leases)) { return false; } else { -// leases.stream().filter(lease -> lease.checkpoint().isShardEnd()) - return false; + List hashRangesForActiveLeases = leases.stream() + .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()) + .map(lease -> lease.hashKeyRangeForLease()) + .collect(Collectors.toList()); + return !checkForHoleInHashKeyRanges(hashRangesForActiveLeases, MIN_HASH_KEY, MAX_HASH_KEY).isPresent(); + } + } + + @VisibleForTesting + static Optional checkForHoleInHashKeyRanges(List hashKeyRanges, + BigInteger minHashKey, BigInteger maxHashKey) { + List mergedHashKeyRanges = sortAndMergeOverlappingHashRanges(hashKeyRanges); + + if (!mergedHashKeyRanges.get(0).startingHashKey().equals(minHashKey) || !mergedHashKeyRanges + .get(mergedHashKeyRanges.size() - 1).endingHashKey().equals(maxHashKey)) { + log.error("Incomplete hash range found between {} and {}.", mergedHashKeyRanges.get(0), + mergedHashKeyRanges.get(mergedHashKeyRanges.size() - 1)); + return Optional.of(new HashRangeHole(mergedHashKeyRanges.get(0), + mergedHashKeyRanges.get(mergedHashKeyRanges.size() - 1))); + } + if (mergedHashKeyRanges.size() > 1) { + for (int i = 1; i < mergedHashKeyRanges.size(); i++) { + final HashKeyRangeForLease hashRangeAtStartOfPossibleHole = mergedHashKeyRanges.get(i - 1); + final HashKeyRangeForLease hashRangeAtEndOfPossibleHole = mergedHashKeyRanges.get(i); + final BigInteger startOfPossibleHole = hashRangeAtStartOfPossibleHole.endingHashKey(); + final BigInteger endOfPossibleHole = hashRangeAtEndOfPossibleHole.startingHashKey(); + + if (!endOfPossibleHole.subtract(startOfPossibleHole).equals(BigInteger.ONE)) { + log.error("Incomplete hash range found between {} and {}.", hashRangeAtStartOfPossibleHole, + hashRangeAtEndOfPossibleHole); + return Optional.of(new HashRangeHole(hashRangeAtStartOfPossibleHole, hashRangeAtEndOfPossibleHole)); + } + } + } + return Optional.empty(); + } + + @VisibleForTesting + static List sortAndMergeOverlappingHashRanges(List hashKeyRanges) { + if(hashKeyRanges.size() == 0 || hashKeyRanges.size() == 1) + return hashKeyRanges; + + Collections.sort(hashKeyRanges, new HashKeyRangeComparator()); + + final HashKeyRangeForLease first = hashKeyRanges.get(0); + BigInteger start = first.startingHashKey(); + BigInteger end = first.endingHashKey(); + + final List result = new ArrayList<>(); + + for (int i = 1; i < hashKeyRanges.size(); i++) { + HashKeyRangeForLease current = hashKeyRanges.get(i); + if (current.startingHashKey().compareTo(end) <= 0) { + end = current.endingHashKey().max(end); + } else { + result.add(new HashKeyRangeForLease(start, end)); + start = current.startingHashKey(); + end = current.endingHashKey(); + } + } + result.add(new HashKeyRangeForLease(start, end)); + return result; + } + + @Value + private static class HashRangeHole { + private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole; + private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; + } + + /** + * Helper class to compare leases based on their hash range. + */ + private static class HashKeyRangeComparator implements Comparator, Serializable { + + private static final long serialVersionUID = 1L; + + @Override + public int compare(HashKeyRangeForLease hashKeyRange, HashKeyRangeForLease otherHashKeyRange) { + return hashKeyRange.startingHashKey().compareTo(otherHashKeyRange.startingHashKey()); } } } 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 427b3509..359b7a44 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/Lease.java @@ -307,4 +307,6 @@ public class Lease { public Lease copy() { return new Lease(this); } + + } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java new file mode 100644 index 00000000..2567a00a --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/PeriodicShardSyncManagerTest.java @@ -0,0 +1,155 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.coordinator; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.kinesis.common.HashKeyRangeForLease; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; + +import static software.amazon.kinesis.common.HashKeyRangeForLease.deserialize; + +@RunWith(MockitoJUnitRunner.class) + +public class PeriodicShardSyncManagerTest { + + @Before + public void setup() { + + } + + @Test + public void testIfHashRangesAreNotMergedWhenNoOverlappingIntervalsGiven() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("24", "30")); + }}; + List sortAndMergedHashRanges = PeriodicShardSyncManager + .sortAndMergeOverlappingHashRanges(hashRanges); + Assert.assertEquals(hashRanges, sortAndMergedHashRanges); + } + + @Test + public void testIfHashRangesAreSortedWhenNoOverlappingIntervalsGiven() { + List hashRanges = new ArrayList() {{ + add(deserialize("2", "3")); + add(deserialize("0", "1")); + add(deserialize("24", "30")); + add(deserialize("4", "23")); + }}; + List hashRangesCopy = new ArrayList<>(); + hashRangesCopy.addAll(hashRanges); + List sortAndMergedHashRanges = PeriodicShardSyncManager + .sortAndMergeOverlappingHashRanges(hashRangesCopy); + Assert.assertEquals(hashRangesCopy, sortAndMergedHashRanges); + Assert.assertNotEquals(hashRanges, sortAndMergedHashRanges); + } + + @Test + public void testIfHashRangesAreMergedWhenOverlappingIntervalsGivenCase1() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", "30")); + }}; + List expectedHashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("24", "30")); + }}; + List sortAndMergedHashRanges = PeriodicShardSyncManager + .sortAndMergeOverlappingHashRanges(hashRanges); + Assert.assertEquals(expectedHashRanges, sortAndMergedHashRanges); + } + + @Test + public void testIfHashRangesAreMergedWhenOverlappingIntervalsGivenCase2() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "5")); + add(deserialize("4", "23")); + add(deserialize("24", "30")); + }}; + List expectedHashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("24", "30")); + }}; + List sortAndMergedHashRanges = PeriodicShardSyncManager + .sortAndMergeOverlappingHashRanges(hashRanges); + Assert.assertEquals(expectedHashRanges, sortAndMergedHashRanges); + } + + @Test + public void testIfHashRangesAreMergedWhenOverlappingIntervalsGivenCase3() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("4", "5")); + add(deserialize("24", "30")); + }}; + List expectedHashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("24", "30")); + }}; + List sortAndMergedHashRanges = PeriodicShardSyncManager + .sortAndMergeOverlappingHashRanges(hashRanges); + Assert.assertEquals(expectedHashRanges, sortAndMergedHashRanges); + } + + @Test + public void testForFailureWhenHashRangesAreIncomplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("25", "30")); // Missing interval here + }}; + Assert.assertTrue(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + } + + @Test + public void testForSuccessWhenHashRangesAreComplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", "30")); + }}; + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + } + +} From a6922d9d7eaf1f8df4a7806d9ed0045320b603e0 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Sat, 2 May 2020 03:22:59 -0700 Subject: [PATCH 088/159] Hash range hole confidence check --- .../coordinator/PeriodicShardSyncManager.java | 74 +++++++++++++------ 1 file changed, 53 insertions(+), 21 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 3edec0ec..90c95cf3 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 @@ -17,6 +17,7 @@ package software.amazon.kinesis.coordinator; import com.google.common.annotations.VisibleForTesting; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NonNull; import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; @@ -61,6 +62,8 @@ class PeriodicShardSyncManager { private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 5 * 60 * 1000L; private static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; private static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); + private static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3; + private Map hashRangeHoleTrackerMap = new HashMap<>(); private final String workerId; private final LeaderDecider leaderDecider; @@ -145,6 +148,7 @@ class PeriodicShardSyncManager { try { final Map> streamToLeasesMap = getStreamToLeasesMap(currentStreamConfigMap.keySet()); for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfigEntry.getValue()); if (!shardSyncTaskManager.syncShardAndLeaseInfo()) { log.warn( @@ -160,18 +164,19 @@ class PeriodicShardSyncManager { } } - private Map> getStreamToLeasesMap(final Set streamIdentifiersToFilter) + private Map> getStreamToLeasesMap( + final Set streamIdentifiersToFilter) throws DependencyException, ProvisionedThroughputException, InvalidStateException { final List leases = leaseRefresher.listLeases(); - if(!isMultiStreamingMode) { + if (!isMultiStreamingMode) { Validate.isTrue(streamIdentifiersToFilter.size() == 1); return Collections.singletonMap(streamIdentifiersToFilter.iterator().next(), leases); } else { final Map> streamToLeasesMap = new HashMap<>(); - for(Lease lease : leases) { + for (Lease lease : leases) { StreamIdentifier streamIdentifier = StreamIdentifier .multiStreamInstance(((MultiStreamLease) lease).streamIdentifier()); - if(streamIdentifiersToFilter.contains(streamIdentifier)) { + if (streamIdentifiersToFilter.contains(streamIdentifier)) { streamToLeasesMap.computeIfAbsent(streamIdentifier, s -> new ArrayList<>()).add(lease); } } @@ -179,25 +184,37 @@ class PeriodicShardSyncManager { } } - /** - * Checks if the entire hash range is covered - * @return true if covered, false otherwise - */ - private boolean isHashRangeCompleteForLeases(List leases) { + // TODO : Catch exception + private boolean shouldDoShardSync(StreamIdentifier streamIdentifier, List leases) { if (CollectionUtils.isNullOrEmpty(leases)) { - return false; + throw new IllegalArgumentException("No leases found to validate for the stream " + streamIdentifier); + } + // Check if there are any holes in the leases and return the first hole if present. + Optional hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases); + if (hashRangeHoleOpt.isPresent()) { + // If hole is present, check if the hole is detected consecutively in previous occurrences. + // If hole is determined with high confidence return true; return false otherwise + return hashRangeHoleTrackerMap.computeIfAbsent(streamIdentifier, s -> new HashRangeHoleTracker()) + .hasHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); + } else { - List hashRangesForActiveLeases = leases.stream() - .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()) - .map(lease -> lease.hashKeyRangeForLease()) - .collect(Collectors.toList()); - return !checkForHoleInHashKeyRanges(hashRangesForActiveLeases, MIN_HASH_KEY, MAX_HASH_KEY).isPresent(); + // If hole is not present, clear any previous tracking for this stream and return false; + hashRangeHoleTrackerMap.remove(streamIdentifier); + return false; } } + private Optional hasHoleInLeases(StreamIdentifier streamIdentifier, List leases) { + // Filter the hashranges of leases which has any checkpoint other than shard end. + List hashRangesForActiveLeases = leases.stream() + .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()) + .map(lease -> lease.hashKeyRangeForLease()).collect(Collectors.toList()); + return checkForHoleInHashKeyRanges(streamIdentifier, hashRangesForActiveLeases, MIN_HASH_KEY, MAX_HASH_KEY); + } + @VisibleForTesting - static Optional checkForHoleInHashKeyRanges(List hashKeyRanges, - BigInteger minHashKey, BigInteger maxHashKey) { + static Optional checkForHoleInHashKeyRanges(StreamIdentifier streamIdentifier, + List hashKeyRanges, BigInteger minHashKey, BigInteger maxHashKey) { List mergedHashKeyRanges = sortAndMergeOverlappingHashRanges(hashKeyRanges); if (!mergedHashKeyRanges.get(0).startingHashKey().equals(minHashKey) || !mergedHashKeyRanges @@ -225,8 +242,9 @@ class PeriodicShardSyncManager { } @VisibleForTesting - static List sortAndMergeOverlappingHashRanges(List hashKeyRanges) { - if(hashKeyRanges.size() == 0 || hashKeyRanges.size() == 1) + static List sortAndMergeOverlappingHashRanges( + List hashKeyRanges) { + if (hashKeyRanges.size() == 0 || hashKeyRanges.size() == 1) return hashKeyRanges; Collections.sort(hashKeyRanges, new HashKeyRangeComparator()); @@ -257,6 +275,21 @@ class PeriodicShardSyncManager { private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; } + private static class HashRangeHoleTracker { + private HashRangeHole hashRangeHole; + private Integer numConsecutiveHoles; + + public boolean hasHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) { + if (hashRangeHole.equals(this.hashRangeHole)) { + ++this.numConsecutiveHoles; + } else { + this.hashRangeHole = hashRangeHole; + this.numConsecutiveHoles = 1; + } + return numConsecutiveHoles >= CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; + } + } + /** * Helper class to compare leases based on their hash range. */ @@ -264,8 +297,7 @@ class PeriodicShardSyncManager { private static final long serialVersionUID = 1L; - @Override - public int compare(HashKeyRangeForLease hashKeyRange, HashKeyRangeForLease otherHashKeyRange) { + @Override public int compare(HashKeyRangeForLease hashKeyRange, HashKeyRangeForLease otherHashKeyRange) { return hashKeyRange.startingHashKey().compareTo(otherHashKeyRange.startingHashKey()); } } From 9e97edd2730d6cf756b78e9e76c4a7633a8e486f Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 6 May 2020 11:06:23 -0700 Subject: [PATCH 089/159] Reusing code while determining leaskekey --- .../amazon/kinesis/coordinator/Scheduler.java | 6 ++-- .../lifecycle/BlockOnParentShardTask.java | 3 +- .../amazon/kinesis/lifecycle/ProcessTask.java | 3 +- .../lifecycle/ShardConsumerSubscriber.java | 4 +-- .../kinesis/lifecycle/ShutdownTask.java | 33 +++---------------- 5 files changed, 10 insertions(+), 39 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 d5eb1be6..a2f9a45e 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 @@ -851,8 +851,7 @@ public class Scheduler implements Runnable { if (!firstItem) { builder.append(", "); } - builder.append(shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) - .orElse(shardInfo.shardId())); + builder.append(ShardInfo.getLeaseKey(shardInfo)); firstItem = false; } slog.info("Current stream shard assignments: " + builder.toString()); @@ -948,8 +947,7 @@ public class Scheduler implements Runnable { ShardConsumer consumer = shardInfoShardConsumerMap.get(shard); if (consumer.leaseLost()) { shardInfoShardConsumerMap.remove(shard); - log.debug("Removed consumer for {} as lease has been lost", - shard.streamIdentifierSerOpt().map(s -> s + ":" + shard.shardId()).orElse(shard.shardId())); + log.debug("Removed consumer for {} as lease has been lost", ShardInfo.getLeaseKey(shard)); } else { consumer.executeLifecycle(); } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java index fcb3ffde..5f1ee18c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTask.java @@ -54,8 +54,7 @@ public class BlockOnParentShardTask implements ConsumerTask { @Override public TaskResult call() { Exception exception = null; - final String shardInfoId = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) - .orElse(shardInfo.shardId()); + final String shardInfoId = ShardInfo.getLeaseKey(shardInfo); try { boolean blockedOnParentShard = false; for (String shardId : shardInfo.parentShardIds()) { 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 f576154a..6c52e0de 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ProcessTask.java @@ -76,8 +76,7 @@ public class ProcessTask implements ConsumerTask { @NonNull AggregatorUtil aggregatorUtil, @NonNull MetricsFactory metricsFactory) { this.shardInfo = shardInfo; - this.shardInfoId = shardInfo.streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()) - .orElse(shardInfo.shardId()); + this.shardInfoId = ShardInfo.getLeaseKey(shardInfo); this.shardRecordProcessor = shardRecordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; this.backoffTimeMillis = backoffTimeMillis; 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 177c0f43..21e8c2c9 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerSubscriber.java @@ -24,6 +24,7 @@ import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; +import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.RecordsRetrieved; import software.amazon.kinesis.retrieval.RetryableRetrievalException; @@ -70,8 +71,7 @@ class ShardConsumerSubscriber implements Subscriber { this.bufferSize = bufferSize; this.shardConsumer = shardConsumer; this.readTimeoutsToIgnoreBeforeWarning = readTimeoutsToIgnoreBeforeWarning; - this.shardInfoId = shardConsumer.shardInfo().streamIdentifierSerOpt() - .map(s -> s + ":" + shardConsumer.shardInfo().shardId()).orElse(shardConsumer.shardInfo().shardId()); + this.shardInfoId = ShardInfo.getLeaseKey(shardConsumer.shardInfo()); } 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 9d53e75c..3449b723 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 @@ -15,15 +15,10 @@ package software.amazon.kinesis.lifecycle; import com.google.common.annotations.VisibleForTesting; - -import java.util.List; -import java.util.function.Function; import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.services.kinesis.model.ChildShard; -import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; @@ -47,8 +42,10 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -89,8 +86,8 @@ public class ShutdownTask implements ConsumerTask { private final List childShards; - private static final Function leaseKeyProvider = shardInfo -> shardInfo - .streamIdentifierSerOpt().map(s -> s + ":" + shardInfo.shardId()).orElse(shardInfo.shardId()); + private static final Function leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo); + /* * Invokes ShardRecordProcessor shutdown() API. * (non-Javadoc) @@ -218,26 +215,4 @@ public class ShutdownTask implements ConsumerTask { return reason; } - private boolean isShardInContextParentOfAny(List shards) { - for(Shard shard : shards) { - if (isChildShardOfShardInContext(shard)) { - return true; - } - } - return false; - } - - private boolean isChildShardOfShardInContext(Shard shard) { - return (StringUtils.equals(shard.parentShardId(), shardInfo.shardId()) - || StringUtils.equals(shard.adjacentParentShardId(), shardInfo.shardId())); - } - - private void dropLease() { - Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); - leaseCoordinator.dropLease(currentLease); - if(currentLease != null) { - log.warn("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); - } - } - } From 80cdf0df6f456b6b64d6c19c63506872a39d9bc4 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 6 May 2020 21:12:50 -0700 Subject: [PATCH 090/159] Update Parent lease fix --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 5 ++++- .../software/amazon/kinesis/lifecycle/ShutdownTaskTest.java | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) 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 9d53e75c..5418f1bf 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 @@ -199,7 +199,10 @@ public class ShutdownTask implements ConsumerTask { final Lease updatedLease = currentLease.copy(); updatedLease.childShardIds(childShardIds); - leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, leaseKeyProvider.apply(shardInfo)); + final boolean updateResult = leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, leaseKeyProvider.apply(shardInfo)); + if (!updateResult) { + throw new InvalidStateException("Failed to update parent lease with child shard information for shard " + shardInfo.shardId()); + } log.info("Shard {}: Updated current lease {} with child shard information: {}", shardInfo.shardId(), currentLease.leaseKey(), childShardIds); } 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 7992d604..f65655db 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 @@ -127,11 +127,12 @@ public class ShutdownTaskTest { * This test is for the scenario that customer doesn't implement checkpoint in their implementation */ @Test - public final void testCallWhenApplicationDoesNotCheckpoint() { + public final void testCallWhenApplicationDoesNotCheckpoint() throws Exception { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); Lease heldLease = 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); final TaskResult result = task.call(); assertNotNull(result.getException()); @@ -174,6 +175,7 @@ public class ShutdownTaskTest { Lease heldLease = 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); final TaskResult result = task.call(); assertNull(result.getException()); From 9115f2000b383aeb2fcb372419c53cefeac0fdb0 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 7 May 2020 01:28:56 -0700 Subject: [PATCH 091/159] Logic to auto fill the missing hashranges and lease recovery. Added more unit test cases --- .../coordinator/PeriodicShardSyncManager.java | 116 +++++-- .../amazon/kinesis/coordinator/Scheduler.java | 8 +- .../amazon/kinesis/leases/LeaseRefresher.java | 15 + .../kinesis/leases/LeaseSerializer.java | 10 + .../kinesis/leases/ShardSyncTaskManager.java | 14 +- .../amazon/kinesis/leases/UpdateField.java | 19 ++ .../dynamodb/DynamoDBLeaseRefresher.java | 22 ++ .../dynamodb/DynamoDBLeaseSerializer.java | 23 ++ .../kinesis/lifecycle/ShutdownTask.java | 4 +- .../PeriodicShardSyncManagerTest.java | 296 +++++++++++++++++- .../kinesis/coordinator/SchedulerTest.java | 4 +- 11 files changed, 495 insertions(+), 36 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java 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 90c95cf3..3979236b 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 @@ -21,6 +21,7 @@ import lombok.NonNull; import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.StreamConfig; @@ -28,7 +29,9 @@ import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.MultiStreamLease; +import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardSyncTaskManager; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -50,6 +53,8 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; +import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRange; + /** * The top level orchestrator for coordinating the periodic shard sync related * activities. @@ -59,10 +64,13 @@ import java.util.stream.Collectors; @Slf4j class PeriodicShardSyncManager { private static final long INITIAL_DELAY = 60 * 1000L; - private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 5 * 60 * 1000L; - private static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; - private static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); - private static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3; + private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L; + @VisibleForTesting + static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; + @VisibleForTesting + static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); + @VisibleForTesting + static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3; private Map hashRangeHoleTrackerMap = new HashMap<>(); private final String workerId; @@ -126,7 +134,7 @@ class PeriodicShardSyncManager { log.info("Syncing Kinesis shard info for " + streamIdentifier); final StreamConfig streamConfig = streamConfigEntry.getValue(); final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfig); - final TaskResult taskResult = shardSyncTaskManager.executeShardSyncTask(); + final TaskResult taskResult = shardSyncTaskManager.callShardSyncTask(); if (taskResult.getException() != null) { throw taskResult.getException(); } @@ -145,19 +153,30 @@ class PeriodicShardSyncManager { private void runShardSync() { if (leaderDecider.isLeader(workerId)) { + log.info(String.format("WorkerId %s is leader, running the periodic shard sync task", workerId)); try { - final Map> streamToLeasesMap = getStreamToLeasesMap(currentStreamConfigMap.keySet()); - for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + // Construct the stream to leases map to be used in the lease sync + final Map> streamToLeasesMap = getStreamToLeasesMap( + currentStreamConfigMap.keySet()); - final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider.apply(streamConfigEntry.getValue()); - if (!shardSyncTaskManager.syncShardAndLeaseInfo()) { - log.warn( - "Failed to submit shard sync task for stream {}. This could be due to the previous shard sync task not finished.", - shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + // For each of the stream, check if shard sync needs to be done based on the leases state. + for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { + if (shouldDoShardSync(streamConfigEntry.getKey(), + streamToLeasesMap.get(streamConfigEntry.getKey()))) { + log.info("Periodic shard syncer initiating shard sync for {}", streamConfigEntry.getKey()); + final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider + .apply(streamConfigEntry.getValue()); + if (!shardSyncTaskManager.castShardSyncTask()) { + log.warn( + "Failed to submit shard sync task for stream {}. This could be due to the previous shard sync task not finished.", + shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); + } + } else { + log.info("Skipping shard sync for {} as either hash ranges are complete in the lease table or leases hole confidence is not achieved.", streamConfigEntry.getKey()); } } } catch (Exception e) { - // TODO : Log + log.error("Caught exception while running periodic shard syncer.", e); } } else { log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); @@ -184,10 +203,12 @@ class PeriodicShardSyncManager { } } - // TODO : Catch exception - private boolean shouldDoShardSync(StreamIdentifier streamIdentifier, List leases) { + @VisibleForTesting + boolean shouldDoShardSync(StreamIdentifier streamIdentifier, List leases) { if (CollectionUtils.isNullOrEmpty(leases)) { - throw new IllegalArgumentException("No leases found to validate for the stream " + streamIdentifier); + // If the leases is null or empty then we need to do shard sync + log.info("No leases found for {}. Will be triggering shard sync", streamIdentifier); + return true; } // Check if there are any holes in the leases and return the first hole if present. Optional hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases); @@ -205,25 +226,76 @@ class PeriodicShardSyncManager { } private Optional hasHoleInLeases(StreamIdentifier streamIdentifier, List leases) { - // Filter the hashranges of leases which has any checkpoint other than shard end. - List hashRangesForActiveLeases = leases.stream() - .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()) + // Filter the leases with any checkpoint other than shard end. + List activeLeases = leases.stream() + .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()).collect(Collectors.toList()); + List activeLeasesWithHashRanges = fillWithHashRangesIfRequired(streamIdentifier, activeLeases); + List hashRangesForActiveLeases = activeLeasesWithHashRanges.stream() .map(lease -> lease.hashKeyRangeForLease()).collect(Collectors.toList()); return checkForHoleInHashKeyRanges(streamIdentifier, hashRangesForActiveLeases, MIN_HASH_KEY, MAX_HASH_KEY); } + // If leases are missing hashranges information, update the leases in-memory as well as in the lease storage + // by learning from kinesis shards. + private List fillWithHashRangesIfRequired(StreamIdentifier streamIdentifier, List activeLeases) { + List activeLeasesWithNoHashRanges = activeLeases.stream() + .filter(lease -> lease.hashKeyRangeForLease() == null).collect(Collectors.toList()); + Optional minLeaseOpt = activeLeasesWithNoHashRanges.stream().min(Comparator.comparing(Lease::leaseKey)); + if (minLeaseOpt.isPresent()) { + // TODO : use minLease for new ListShards with startingShardId + final Lease minLease = minLeaseOpt.get(); + final ShardDetector shardDetector = shardSyncTaskManagerProvider + .apply(currentStreamConfigMap.get(streamIdentifier)).shardDetector(); + final Map kinesisShards = shardDetector.listShards().stream() + .collect(Collectors.toMap(Shard::shardId, shard -> shard)); + return activeLeases.stream().map(lease -> { + if (lease.hashKeyRangeForLease() == null) { + final String shardId = lease instanceof MultiStreamLease ? + ((MultiStreamLease) lease).shardId() : + lease.leaseKey(); + final Shard shard = kinesisShards.get(shardId); + if(shard == null) { + return lease; + } + lease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); + try { + leaseRefresher.updateLease(lease, UpdateField.HASH_KEY_RANGE); + } catch (Exception e) { + log.warn( + "Unable to update hash range key information for lease {} of stream {}. This may result in explicit lease sync.", + lease.leaseKey(), streamIdentifier); + } + } + return lease; + }).filter(lease -> lease.hashKeyRangeForLease() != null).collect(Collectors.toList()); + } else { + return activeLeases; + } + } + @VisibleForTesting static Optional checkForHoleInHashKeyRanges(StreamIdentifier streamIdentifier, List hashKeyRanges, BigInteger minHashKey, BigInteger maxHashKey) { + // Sort and merge the overlapping hash ranges. List mergedHashKeyRanges = sortAndMergeOverlappingHashRanges(hashKeyRanges); + if(mergedHashKeyRanges.isEmpty()) { + log.error("No valid hashranges found for stream {} between {} and {}.", streamIdentifier, + MIN_HASH_KEY, MAX_HASH_KEY); + return Optional.of(new HashRangeHole(new HashKeyRangeForLease(MIN_HASH_KEY, MAX_HASH_KEY), + new HashKeyRangeForLease(MIN_HASH_KEY, MAX_HASH_KEY))); + } + + // Validate for hashranges bounds. if (!mergedHashKeyRanges.get(0).startingHashKey().equals(minHashKey) || !mergedHashKeyRanges .get(mergedHashKeyRanges.size() - 1).endingHashKey().equals(maxHashKey)) { - log.error("Incomplete hash range found between {} and {}.", mergedHashKeyRanges.get(0), + log.error("Incomplete hash range found for stream {} between {} and {}.", streamIdentifier, + mergedHashKeyRanges.get(0), mergedHashKeyRanges.get(mergedHashKeyRanges.size() - 1)); return Optional.of(new HashRangeHole(mergedHashKeyRanges.get(0), mergedHashKeyRanges.get(mergedHashKeyRanges.size() - 1))); } + // Check for any holes in the sorted hashrange intervals. if (mergedHashKeyRanges.size() > 1) { for (int i = 1; i < mergedHashKeyRanges.size(); i++) { final HashKeyRangeForLease hashRangeAtStartOfPossibleHole = mergedHashKeyRanges.get(i - 1); @@ -232,8 +304,8 @@ class PeriodicShardSyncManager { final BigInteger endOfPossibleHole = hashRangeAtEndOfPossibleHole.startingHashKey(); if (!endOfPossibleHole.subtract(startOfPossibleHole).equals(BigInteger.ONE)) { - log.error("Incomplete hash range found between {} and {}.", hashRangeAtStartOfPossibleHole, - hashRangeAtEndOfPossibleHole); + log.error("Incomplete hash range found for {} between {} and {}.", streamIdentifier, + hashRangeAtStartOfPossibleHole, hashRangeAtEndOfPossibleHole); return Optional.of(new HashRangeHole(hashRangeAtStartOfPossibleHole, hashRangeAtEndOfPossibleHole)); } } 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 a2f9a45e..3b1003ee 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 @@ -351,7 +351,7 @@ public class Scheduler implements Runnable { log.info("LeaseCoordinator is already running. No need to start it."); } log.info("Scheduling periodicShardSync"); - // leaderElectedPeriodicShardSyncManager.start(shardSyncTasks); + leaderElectedPeriodicShardSyncManager.start(); // TODO: enable periodicShardSync after https://github.com/jushkem/amazon-kinesis-client/pull/2 is merged // TODO: Determine if waitUntilHashRangeCovered() is needed. streamSyncWatch.start(); @@ -417,7 +417,7 @@ public class Scheduler implements Runnable { final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifierSerOpt()); final StreamConfig streamConfig = currentStreamConfigMap .getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); - if (createOrGetShardSyncTaskManager(streamConfig).syncShardAndLeaseInfo()) { + if (createOrGetShardSyncTaskManager(streamConfig).castShardSyncTask()) { log.info("{} : Found completed shard, initiated new ShardSyncTak for {} ", streamIdentifier.serialize(), completedShard.toString()); } @@ -480,7 +480,7 @@ public class Scheduler implements Runnable { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.syncShardAndLeaseInfo(); + shardSyncTaskManager.castShardSyncTask(); currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); streamsSynced.add(streamIdentifier); } else { @@ -508,7 +508,7 @@ public class Scheduler implements Runnable { + ". Syncing shards of that stream."); ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager( currentStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.syncShardAndLeaseInfo(); + shardSyncTaskManager.castShardSyncTask(); currentSetOfStreamsIter.remove(); streamsSynced.add(streamIdentifier); } 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 0d563a63..4ba0cf86 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 @@ -191,6 +191,21 @@ public interface LeaseRefresher { boolean updateLease(Lease lease) throws DependencyException, InvalidStateException, ProvisionedThroughputException; + /** + * Update application-specific fields of the given lease in DynamoDB. Does not update fields managed by the leasing + * library such as leaseCounter, leaseOwner, or leaseKey. + * + * @return true if update succeeded, false otherwise + * + * @throws InvalidStateException if lease table does not exist + * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity + * @throws DependencyException if DynamoDB update fails in an unexpected way + */ + default void updateLease(Lease lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throw new UnsupportedOperationException("updateLeaseWithNoExpectation is not implemented"); + } + /** * Check (synchronously) if there are any leases in the lease table. * diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java index 95b98399..5dbf6366 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java @@ -107,6 +107,15 @@ public interface LeaseSerializer { */ Map getDynamoUpdateLeaseUpdate(Lease lease); + /** + * @param lease + * @param updateField + * @return the attribute value map that updates application-specific data for a lease + */ + default Map getDynamoUpdateLeaseUpdate(Lease lease, UpdateField updateField) { + throw new UnsupportedOperationException(); + } + /** * @return the key schema for creating a DynamoDB table to store leases */ @@ -116,4 +125,5 @@ public interface LeaseSerializer { * @return attribute definitions for creating a DynamoDB table to store leases */ Collection getAttributeDefinitions(); + } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java index a52ac650..de3d4c6c 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 @@ -126,7 +126,11 @@ public class ShardSyncTaskManager { this.lock = new ReentrantLock(); } - public TaskResult executeShardSyncTask() { + /** + * Call a ShardSyncTask and return the Task Result. + * @return the Task Result. + */ + public TaskResult callShardSyncTask() { final ShardSyncTask shardSyncTask = new ShardSyncTask(shardDetector, leaseRefresher, initialPositionInStream, @@ -140,7 +144,11 @@ public class ShardSyncTaskManager { return metricCollectingTask.call(); } - public boolean syncShardAndLeaseInfo() { + /** + * Cast a ShardSyncTask and return if the casting is successful. + * @return if the casting is successful. + */ + public boolean castShardSyncTask() { try { lock.lock(); return checkAndSubmitNextTask(); @@ -197,7 +205,7 @@ public class ShardSyncTaskManager { log.error("Caught exception running {} task: ", currentTask.taskType(), exception != null ? exception : taskResult.getException()); } // Acquire lock here. If shardSyncRequestPending is false in this completionStage and - // syncShardAndLeaseInfo is invoked, before completion stage exits (future completes) + // castShardSyncTask is invoked, before completion stage exits (future completes) // but right after the value of shardSyncRequestPending is checked, it will result in // shardSyncRequestPending being set to true, but no pending futures to trigger the next // ShardSyncTask. By executing this stage in a Reentrant lock, we ensure that if the diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java new file mode 100644 index 00000000..c15449ca --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java @@ -0,0 +1,19 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package software.amazon.kinesis.leases; + +public enum UpdateField { + CHILD_SHARDS, HASH_KEY_RANGE +} 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 67e5abbe..867cc507 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 @@ -35,6 +35,7 @@ import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.LeaseSerializer; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; @@ -659,6 +660,27 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { return true; } + @Override + public void updateLease(Lease lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + log.debug("Updating lease without expectation {}", lease); + final AWSExceptionManager exceptionManager = createExceptionManager(); + Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); + UpdateItemRequest request = UpdateItemRequest.builder().tableName(table).key(serializer.getDynamoHashKey(lease)) + .attributeUpdates(updates).build(); + try { + try { + FutureUtils.resolveOrCancelFuture(dynamoDBClient.updateItem(request), dynamoDbRequestTimeout); + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } catch (InterruptedException e) { + throw new DependencyException(e); + } + } catch (DynamoDbException | TimeoutException e) { + throw convertAndRethrowExceptions("update", lease.leaseKey(), e); + } + } + /** * {@inheritDoc} */ 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 8f293881..4523bada 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 @@ -36,6 +36,7 @@ import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.leases.DynamoUtils; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseSerializer; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; /** @@ -268,6 +269,28 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { return result; } + @Override + public Map getDynamoUpdateLeaseUpdate(Lease lease, + UpdateField updateField) { + Map result = new HashMap<>(); + switch (updateField) { + case CHILD_SHARDS: + if (!CollectionUtils.isNullOrEmpty(lease.childShardIds())) { + result.put(CHILD_SHARD_IDS_KEY, putUpdate(DynamoUtils.createAttributeValue(lease.childShardIds()))); + } + break; + case HASH_KEY_RANGE: + if (lease.hashKeyRangeForLease() != null) { + result.put(STARTING_HASH_KEY, putUpdate( + DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedStartingHashKey()))); + result.put(ENDING_HASH_KEY, putUpdate( + DynamoUtils.createAttributeValue(lease.hashKeyRangeForLease().serializedEndingHashKey()))); + } + break; + } + return result; + } + @Override public Collection getKeySchema() { List keySchema = new ArrayList<>(); 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 3449b723..2544eac6 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 @@ -115,7 +115,7 @@ public class ShutdownTask implements ConsumerTask { // This scenario could happen when customer deletes the stream while leaving the KCL application running. if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); - updateLeasesForChildShards(); + updateLeasesWithChildShards(); } else { log.warn("Shard {} no longer exists. Shutting down consumer with SHARD_END reason without creating leases for child shards.", leaseKeyProvider.apply(shardInfo)); } @@ -189,7 +189,7 @@ public class ShutdownTask implements ConsumerTask { } } - private void updateLeasesForChildShards() + private void updateLeasesWithChildShards() throws DependencyException, InvalidStateException, ProvisionedThroughputException { final Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); Set childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet()); 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 2567a00a..eb88c0dc 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 @@ -19,22 +19,56 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; +import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.HashKeyRange; +import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.common.HashKeyRangeForLease; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseRefresher; +import software.amazon.kinesis.leases.MultiStreamLease; +import software.amazon.kinesis.leases.ShardDetector; +import software.amazon.kinesis.leases.ShardSyncTaskManager; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.math.BigInteger; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static software.amazon.kinesis.common.HashKeyRangeForLease.deserialize; +import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; +import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MAX_HASH_KEY; +import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MIN_HASH_KEY; @RunWith(MockitoJUnitRunner.class) public class PeriodicShardSyncManagerTest { + private StreamIdentifier streamIdentifier; + private PeriodicShardSyncManager periodicShardSyncManager; + @Mock + private LeaderDecider leaderDecider; + @Mock + private LeaseRefresher leaseRefresher; + @Mock + Map currentStreamConfigMap; + @Mock + Function shardSyncTaskManagerProvider; + @Before public void setup() { - + streamIdentifier = StreamIdentifier.multiStreamInstance("123:stream:456"); + periodicShardSyncManager = new PeriodicShardSyncManager("worker", leaderDecider, leaseRefresher, currentStreamConfigMap, + shardSyncTaskManagerProvider, true); } @Test @@ -136,7 +170,7 @@ public class PeriodicShardSyncManagerTest { add(deserialize("25", "30")); // Missing interval here }}; Assert.assertTrue(PeriodicShardSyncManager - .checkForHoleInHashKeyRanges(hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); } @Test @@ -149,7 +183,263 @@ public class PeriodicShardSyncManagerTest { add(deserialize("24", "30")); }}; Assert.assertFalse(PeriodicShardSyncManager - .checkForHoleInHashKeyRanges(hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() { + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, null)); + } + + @Test + public void testIfShardSyncIsInitiatedWhenEmptyLeasesArePassed() { + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, new ArrayList<>())); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenConfidenceFactorIsNotReached() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenConfidenceFactorIsReached() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + } + + @Test + public void testIfShardSyncIsInitiatedWhenHoleIsDueToShardEnd() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); // introducing hole here through SHARD_END checkpoint + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + if(lease.hashKeyRangeForLease().startingHashKey().toString().equals("4")) { + lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + } else { + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + } + + @Test + public void testIfShardSyncIsInitiatedWhenNoLeasesAreUsedDueToShardEnd() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.SHARD_END); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenHoleShifts() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + List multiStreamLeases2 = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); // Hole between 3 and 5 + add(deserialize("5", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + // Resetting the holes + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases2))); + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases2)); + } + + @Test + public void testIfShardSyncIsNotInitiatedWhenHoleShiftsMoreThanOnce() { + List multiStreamLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); // Hole between 23 and 25 + add(deserialize("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + List multiStreamLeases2 = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); // Hole between 3 and 5 + add(deserialize("5", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + // Resetting the holes + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases2))); + // Resetting the holes + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + } + + @Test + public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSync() { + ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); + ShardDetector shardDetector = mock(ShardDetector.class); + when(shardSyncTaskManagerProvider.apply(any())).thenReturn(shardSyncTaskManager); + when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + + final int[] shardCounter = { 0 }; + List hashKeyRangeForLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "20")); + add(deserialize("21", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}; + + List kinesisShards = hashKeyRangeForLeases.stream() + .map(hashKeyRangeForLease -> Shard.builder().shardId("shard-" + (++shardCounter[0])).hashKeyRange( + HashKeyRange.builder().startingHashKey(hashKeyRangeForLease.serializedStartingHashKey()) + .endingHashKey(hashKeyRangeForLease.serializedEndingHashKey()).build()).build()) + .collect(Collectors.toList()); + + when(shardDetector.listShards()).thenReturn(kinesisShards); + + final int[] leaseCounter = { 0 }; + List multiStreamLeases = hashKeyRangeForLeases.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0]))); + lease.shardId("shard-"+(leaseCounter[0])); + // Setting the hashrange only for last two leases + if(leaseCounter[0] >= 3) { + lease.hashKeyRange(hashKeyRangeForLease); + } + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + // Assert that shard sync should never trigger + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + Assert.assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + + // Assert that all the leases now has hashRanges set. + for(Lease lease : multiStreamLeases) { + Assert.assertNotNull(lease.hashKeyRangeForLease()); + } + } + + @Test + public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSyncInHoleScenario() { + ShardSyncTaskManager shardSyncTaskManager = mock(ShardSyncTaskManager.class); + ShardDetector shardDetector = mock(ShardDetector.class); + when(shardSyncTaskManagerProvider.apply(any())).thenReturn(shardSyncTaskManager); + when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + + final int[] shardCounter = { 0 }; + List hashKeyRangeForLeases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("5", "20")); // Hole between 3 and 5 + add(deserialize("21", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}; + + List kinesisShards = hashKeyRangeForLeases.stream() + .map(hashKeyRangeForLease -> Shard.builder().shardId("shard-" + (++shardCounter[0])).hashKeyRange( + HashKeyRange.builder().startingHashKey(hashKeyRangeForLease.serializedStartingHashKey()) + .endingHashKey(hashKeyRangeForLease.serializedEndingHashKey()).build()).build()) + .collect(Collectors.toList()); + + when(shardDetector.listShards()).thenReturn(kinesisShards); + + final int[] leaseCounter = { 0 }; + List multiStreamLeases = hashKeyRangeForLeases.stream().map(hashKeyRangeForLease -> { + MultiStreamLease lease = new MultiStreamLease(); + lease.leaseKey(MultiStreamLease.getLeaseKey(streamIdentifier.serialize(), "shard-"+(++leaseCounter[0]))); + lease.shardId("shard-"+(leaseCounter[0])); + // Setting the hashrange only for last two leases + if(leaseCounter[0] >= 3) { + lease.hashKeyRange(hashKeyRangeForLease); + } + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + // Assert that shard sync should never trigger + IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + + // Assert that all the leases now has hashRanges set. + for(Lease lease : multiStreamLeases) { + Assert.assertNotNull(lease.hashKeyRangeForLease()); + } } } 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 a5fd2add..e5a76ce3 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 @@ -190,7 +190,7 @@ public class SchedulerTest { }); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); - when(shardSyncTaskManager.executeShardSyncTask()).thenReturn(new TaskResult(null)); + when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); when(retrievalFactory.createGetRecordsCache(any(ShardInfo.class), any(MetricsFactory.class))).thenReturn(recordsPublisher); when(shardDetector.streamIdentifier()).thenReturn(mock(StreamIdentifier.class)); @@ -1036,7 +1036,7 @@ public class SchedulerTest { shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); when(shardDetector.streamIdentifier()).thenReturn(streamConfig.streamIdentifier()); - when(shardSyncTaskManager.executeShardSyncTask()).thenReturn(new TaskResult(null)); + when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); if(shardSyncFirstAttemptFailure) { when(shardDetector.listShards()) .thenThrow(new RuntimeException("Service Exception")) From 02ea8cd70ffe87b4dbbb8e12ec58bc0c2b55a33e Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 7 May 2020 01:36:51 -0700 Subject: [PATCH 092/159] Added TODO for childshards update fix --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 1 + 1 file changed, 1 insertion(+) 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 2544eac6..9e295616 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 @@ -196,6 +196,7 @@ public class ShutdownTask implements ConsumerTask { final Lease updatedLease = currentLease.copy(); updatedLease.childShardIds(childShardIds); + // TODO : the following update will silently fail. Make changes to use the new leaserefresher#updateLease(Lease lease, UpdateField updateField) leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, leaseKeyProvider.apply(shardInfo)); log.info("Shard {}: Updated current lease {} with child shard information: {}", shardInfo.shardId(), currentLease.leaseKey(), childShardIds); } From fc4781e3475303360fead56a1bc3b9372081a160 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 21 May 2020 00:57:24 -0700 Subject: [PATCH 093/159] Addressed review comments --- .../coordinator/PeriodicShardSyncManager.java | 165 ++++++++++-------- .../amazon/kinesis/coordinator/Scheduler.java | 6 +- .../amazon/kinesis/leases/LeaseRefresher.java | 2 +- .../kinesis/leases/ShardSyncTaskManager.java | 6 +- .../amazon/kinesis/leases/UpdateField.java | 7 + .../dynamodb/DynamoDBLeaseRefresher.java | 2 +- .../PeriodicShardSyncManagerTest.java | 156 ++++------------- 7 files changed, 146 insertions(+), 198 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 3979236b..a96bf01e 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 @@ -19,6 +19,7 @@ import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NonNull; import lombok.Value; +import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; import software.amazon.awssdk.services.kinesis.model.Shard; @@ -161,25 +162,28 @@ class PeriodicShardSyncManager { // For each of the stream, check if shard sync needs to be done based on the leases state. for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { - if (shouldDoShardSync(streamConfigEntry.getKey(), - streamToLeasesMap.get(streamConfigEntry.getKey()))) { - log.info("Periodic shard syncer initiating shard sync for {}", streamConfigEntry.getKey()); + final ShardSyncResponse shardSyncResponse = checkForShardSync(streamConfigEntry.getKey(), + streamToLeasesMap.get(streamConfigEntry.getKey())); + if (shardSyncResponse.shouldDoShardSync()) { + log.info("Periodic shard syncer initiating shard sync for {} due to the reason - ", + streamConfigEntry.getKey(), shardSyncResponse.reasonForDecision()); final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider .apply(streamConfigEntry.getValue()); - if (!shardSyncTaskManager.castShardSyncTask()) { + if (!shardSyncTaskManager.submitShardSyncTask()) { log.warn( - "Failed to submit shard sync task for stream {}. This could be due to the previous shard sync task not finished.", + "Failed to submit shard sync task for stream {}. This could be due to the previous pending shard sync task.", shardSyncTaskManager.shardDetector().streamIdentifier().streamName()); } } else { - log.info("Skipping shard sync for {} as either hash ranges are complete in the lease table or leases hole confidence is not achieved.", streamConfigEntry.getKey()); + log.info("Skipping shard sync for {} due to the reason - {}", streamConfigEntry.getKey(), + shardSyncResponse.reasonForDecision()); } } } catch (Exception e) { log.error("Caught exception while running periodic shard syncer.", e); } } else { - log.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); + log.debug("WorkerId {} is not a leader, not running the shard sync task", workerId); } } @@ -204,35 +208,49 @@ class PeriodicShardSyncManager { } @VisibleForTesting - boolean shouldDoShardSync(StreamIdentifier streamIdentifier, List leases) { + ShardSyncResponse checkForShardSync(StreamIdentifier streamIdentifier, List leases) { if (CollectionUtils.isNullOrEmpty(leases)) { // If the leases is null or empty then we need to do shard sync log.info("No leases found for {}. Will be triggering shard sync", streamIdentifier); - return true; + return new ShardSyncResponse(true, "No leases found for " + streamIdentifier); } // Check if there are any holes in the leases and return the first hole if present. Optional hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases); if (hashRangeHoleOpt.isPresent()) { // If hole is present, check if the hole is detected consecutively in previous occurrences. // If hole is determined with high confidence return true; return false otherwise - return hashRangeHoleTrackerMap.computeIfAbsent(streamIdentifier, s -> new HashRangeHoleTracker()) + // We are using the high confidence factor to avoid shard sync on any holes during resharding and + // lease cleanups or any intermittent issues. + final HashRangeHoleTracker hashRangeHoleTracker = hashRangeHoleTrackerMap + .computeIfAbsent(streamIdentifier, s -> new HashRangeHoleTracker()); + final boolean hasHoleWithHighConfidence = hashRangeHoleTracker .hasHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); + return new ShardSyncResponse(hasHoleWithHighConfidence, + "Detected same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + + " times. Shard sync will be initiated when threshold breaches " + + CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY); } else { // If hole is not present, clear any previous tracking for this stream and return false; hashRangeHoleTrackerMap.remove(streamIdentifier); - return false; + return new ShardSyncResponse(false, "Hash Ranges are complete for " + streamIdentifier); } } + @Value + @Accessors(fluent = true) + @VisibleForTesting + static class ShardSyncResponse { + private final boolean shouldDoShardSync; + private final String reasonForDecision; + } + private Optional hasHoleInLeases(StreamIdentifier streamIdentifier, List leases) { // Filter the leases with any checkpoint other than shard end. List activeLeases = leases.stream() .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()).collect(Collectors.toList()); List activeLeasesWithHashRanges = fillWithHashRangesIfRequired(streamIdentifier, activeLeases); - List hashRangesForActiveLeases = activeLeasesWithHashRanges.stream() - .map(lease -> lease.hashKeyRangeForLease()).collect(Collectors.toList()); - return checkForHoleInHashKeyRanges(streamIdentifier, hashRangesForActiveLeases, MIN_HASH_KEY, MAX_HASH_KEY); + return checkForHoleInHashKeyRanges(streamIdentifier, activeLeasesWithHashRanges); } // If leases are missing hashranges information, update the leases in-memory as well as in the lease storage @@ -259,7 +277,7 @@ class PeriodicShardSyncManager { } lease.hashKeyRange(fromHashKeyRange(shard.hashKeyRange())); try { - leaseRefresher.updateLease(lease, UpdateField.HASH_KEY_RANGE); + leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); } catch (Exception e) { log.warn( "Unable to update hash range key information for lease {} of stream {}. This may result in explicit lease sync.", @@ -275,38 +293,46 @@ class PeriodicShardSyncManager { @VisibleForTesting static Optional checkForHoleInHashKeyRanges(StreamIdentifier streamIdentifier, - List hashKeyRanges, BigInteger minHashKey, BigInteger maxHashKey) { - // Sort and merge the overlapping hash ranges. - List mergedHashKeyRanges = sortAndMergeOverlappingHashRanges(hashKeyRanges); - - if(mergedHashKeyRanges.isEmpty()) { - log.error("No valid hashranges found for stream {} between {} and {}.", streamIdentifier, - MIN_HASH_KEY, MAX_HASH_KEY); - return Optional.of(new HashRangeHole(new HashKeyRangeForLease(MIN_HASH_KEY, MAX_HASH_KEY), - new HashKeyRangeForLease(MIN_HASH_KEY, MAX_HASH_KEY))); + List leasesWithHashKeyRanges) { + // Sort the hash ranges by starting hash key. + List sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges); + if(sortedLeasesWithHashKeyRanges.isEmpty()) { + log.error("No leases with valid hashranges found for stream {}", streamIdentifier); + return Optional.of(new HashRangeHole()); } - // Validate for hashranges bounds. - if (!mergedHashKeyRanges.get(0).startingHashKey().equals(minHashKey) || !mergedHashKeyRanges - .get(mergedHashKeyRanges.size() - 1).endingHashKey().equals(maxHashKey)) { + if (!sortedLeasesWithHashKeyRanges.get(0).hashKeyRangeForLease().startingHashKey().equals(MIN_HASH_KEY) || !sortedLeasesWithHashKeyRanges + .get(sortedLeasesWithHashKeyRanges.size() - 1).hashKeyRangeForLease().endingHashKey().equals(MAX_HASH_KEY)) { log.error("Incomplete hash range found for stream {} between {} and {}.", streamIdentifier, - mergedHashKeyRanges.get(0), - mergedHashKeyRanges.get(mergedHashKeyRanges.size() - 1)); - return Optional.of(new HashRangeHole(mergedHashKeyRanges.get(0), - mergedHashKeyRanges.get(mergedHashKeyRanges.size() - 1))); + sortedLeasesWithHashKeyRanges.get(0), + sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1)); + return Optional.of(new HashRangeHole(sortedLeasesWithHashKeyRanges.get(0), + sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1))); } // Check for any holes in the sorted hashrange intervals. - if (mergedHashKeyRanges.size() > 1) { - for (int i = 1; i < mergedHashKeyRanges.size(); i++) { - final HashKeyRangeForLease hashRangeAtStartOfPossibleHole = mergedHashKeyRanges.get(i - 1); - final HashKeyRangeForLease hashRangeAtEndOfPossibleHole = mergedHashKeyRanges.get(i); - final BigInteger startOfPossibleHole = hashRangeAtStartOfPossibleHole.endingHashKey(); - final BigInteger endOfPossibleHole = hashRangeAtEndOfPossibleHole.startingHashKey(); - - if (!endOfPossibleHole.subtract(startOfPossibleHole).equals(BigInteger.ONE)) { - log.error("Incomplete hash range found for {} between {} and {}.", streamIdentifier, - hashRangeAtStartOfPossibleHole, hashRangeAtEndOfPossibleHole); - return Optional.of(new HashRangeHole(hashRangeAtStartOfPossibleHole, hashRangeAtEndOfPossibleHole)); + if (sortedLeasesWithHashKeyRanges.size() > 1) { + Lease leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(0); + HashKeyRangeForLease leftLeaseHashRange = leftMostLeaseToReportInCaseOfHole.hashKeyRangeForLease(); + for (int i = 1; i < sortedLeasesWithHashKeyRanges.size(); i++) { + final HashKeyRangeForLease rightLeaseHashRange = sortedLeasesWithHashKeyRanges.get(i).hashKeyRangeForLease(); + final BigInteger rangeDiff = rightLeaseHashRange.startingHashKey().subtract(leftLeaseHashRange.endingHashKey()); + // Case of overlapping leases when the rangediff is 0 or negative. + // signum() will be -1 for negative and 0 if value is 0. + // Merge the range for further tracking. + if (rangeDiff.signum() <= 0) { + leftLeaseHashRange = new HashKeyRangeForLease(leftLeaseHashRange.startingHashKey(), + leftLeaseHashRange.endingHashKey().max(rightLeaseHashRange.endingHashKey())); + } else { + // Case of non overlapping leases when rangediff is positive. signum() will be 1 for positive. + // If rangeDiff is 1, then it is a case of continuous hashrange. If not, it is a hole. + if (!rangeDiff.equals(BigInteger.ONE)) { + log.error("Incomplete hash range found for {} between {} and {}.", streamIdentifier, + leftMostLeaseToReportInCaseOfHole, sortedLeasesWithHashKeyRanges.get(i)); + return Optional.of(new HashRangeHole(leftMostLeaseToReportInCaseOfHole, + sortedLeasesWithHashKeyRanges.get(i))); + } + leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(i); + leftLeaseHashRange = rightLeaseHashRange; } } } @@ -314,41 +340,31 @@ class PeriodicShardSyncManager { } @VisibleForTesting - static List sortAndMergeOverlappingHashRanges( - List hashKeyRanges) { - if (hashKeyRanges.size() == 0 || hashKeyRanges.size() == 1) - return hashKeyRanges; - - Collections.sort(hashKeyRanges, new HashKeyRangeComparator()); - - final HashKeyRangeForLease first = hashKeyRanges.get(0); - BigInteger start = first.startingHashKey(); - BigInteger end = first.endingHashKey(); - - final List result = new ArrayList<>(); - - for (int i = 1; i < hashKeyRanges.size(); i++) { - HashKeyRangeForLease current = hashKeyRanges.get(i); - if (current.startingHashKey().compareTo(end) <= 0) { - end = current.endingHashKey().max(end); - } else { - result.add(new HashKeyRangeForLease(start, end)); - start = current.startingHashKey(); - end = current.endingHashKey(); - } - } - result.add(new HashKeyRangeForLease(start, end)); - return result; + static List sortLeasesByHashRange(List leasesWithHashKeyRanges) { + if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1) + return leasesWithHashKeyRanges; + Collections.sort(leasesWithHashKeyRanges, new HashKeyRangeComparator()); + return leasesWithHashKeyRanges; } @Value private static class HashRangeHole { - private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole; - private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; + HashRangeHole() { + leaseAtEndOfPossibleHole = leaseAtStartOfPossibleHole = null; + } + + HashRangeHole(Lease leaseAtStartOfPossibleHole, Lease leaseAtEndOfPossibleHole) { + this.leaseAtStartOfPossibleHole = leaseAtStartOfPossibleHole; + this.leaseAtEndOfPossibleHole = leaseAtEndOfPossibleHole; + } + + private final Lease leaseAtStartOfPossibleHole; + private final Lease leaseAtEndOfPossibleHole; } private static class HashRangeHoleTracker { private HashRangeHole hashRangeHole; + @Getter private Integer numConsecutiveHoles; public boolean hasHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) { @@ -365,12 +381,17 @@ class PeriodicShardSyncManager { /** * Helper class to compare leases based on their hash range. */ - private static class HashKeyRangeComparator implements Comparator, Serializable { + private static class HashKeyRangeComparator implements Comparator, Serializable { private static final long serialVersionUID = 1L; - @Override public int compare(HashKeyRangeForLease hashKeyRange, HashKeyRangeForLease otherHashKeyRange) { - return hashKeyRange.startingHashKey().compareTo(otherHashKeyRange.startingHashKey()); + @Override public int compare(Lease lease, Lease otherLease) { + Validate.notNull(lease); + Validate.notNull(otherLease); + Validate.notNull(lease.hashKeyRangeForLease()); + Validate.notNull(otherLease.hashKeyRangeForLease()); + return lease.hashKeyRangeForLease().startingHashKey() + .compareTo(otherLease.hashKeyRangeForLease().startingHashKey()); } } } 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 3b1003ee..91125a06 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 @@ -417,7 +417,7 @@ public class Scheduler implements Runnable { final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifierSerOpt()); final StreamConfig streamConfig = currentStreamConfigMap .getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); - if (createOrGetShardSyncTaskManager(streamConfig).castShardSyncTask()) { + if (createOrGetShardSyncTaskManager(streamConfig).submitShardSyncTask()) { log.info("{} : Found completed shard, initiated new ShardSyncTak for {} ", streamIdentifier.serialize(), completedShard.toString()); } @@ -480,7 +480,7 @@ public class Scheduler implements Runnable { if (!currentStreamConfigMap.containsKey(streamIdentifier)) { log.info("Found new stream to process: " + streamIdentifier + ". Syncing shards of that stream."); ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager(newStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.castShardSyncTask(); + shardSyncTaskManager.submitShardSyncTask(); currentStreamConfigMap.put(streamIdentifier, newStreamConfigMap.get(streamIdentifier)); streamsSynced.add(streamIdentifier); } else { @@ -508,7 +508,7 @@ public class Scheduler implements Runnable { + ". Syncing shards of that stream."); ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager( currentStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.castShardSyncTask(); + shardSyncTaskManager.submitShardSyncTask(); currentSetOfStreamsIter.remove(); streamsSynced.add(streamIdentifier); } 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 4ba0cf86..b7f38a4e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java @@ -201,7 +201,7 @@ public interface LeaseRefresher { * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws DependencyException if DynamoDB update fails in an unexpected way */ - default void updateLease(Lease lease, UpdateField updateField) + default void updateLeaseWithMetaInfo(Lease lease, UpdateField updateField) throws DependencyException, InvalidStateException, ProvisionedThroughputException { throw new UnsupportedOperationException("updateLeaseWithNoExpectation is not implemented"); } 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 de3d4c6c..6a1ceff4 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 @@ -145,10 +145,10 @@ public class ShardSyncTaskManager { } /** - * Cast a ShardSyncTask and return if the casting is successful. + * Submit a ShardSyncTask and return if the submission is successful. * @return if the casting is successful. */ - public boolean castShardSyncTask() { + public boolean submitShardSyncTask() { try { lock.lock(); return checkAndSubmitNextTask(); @@ -205,7 +205,7 @@ public class ShardSyncTaskManager { log.error("Caught exception running {} task: ", currentTask.taskType(), exception != null ? exception : taskResult.getException()); } // Acquire lock here. If shardSyncRequestPending is false in this completionStage and - // castShardSyncTask is invoked, before completion stage exits (future completes) + // submitShardSyncTask is invoked, before completion stage exits (future completes) // but right after the value of shardSyncRequestPending is checked, it will result in // shardSyncRequestPending being set to true, but no pending futures to trigger the next // ShardSyncTask. By executing this stage in a Reentrant lock, we ensure that if the diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java index c15449ca..9461a18e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/UpdateField.java @@ -14,6 +14,13 @@ */ package software.amazon.kinesis.leases; +/** + * These are the special fields that will be updated only once during the lifetime of the lease. + * Since these are meta information that will not affect lease ownership or data durability, we allow + * any elected leader or worker to set these fields directly without any conditional checks. + * Note that though HASH_KEY_RANGE will be available during lease initialization in newer versions, we keep this + * for backfilling while rolling forward to newer versions. + */ public enum UpdateField { CHILD_SHARDS, HASH_KEY_RANGE } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java index 867cc507..30201236 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 @@ -661,7 +661,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } @Override - public void updateLease(Lease lease, UpdateField updateField) + public void updateLeaseWithMetaInfo(Lease lease, UpdateField updateField) throws DependencyException, InvalidStateException, ProvisionedThroughputException { log.debug("Updating lease without expectation {}", lease); final AWSExceptionManager exceptionManager = createExceptionManager(); 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 eb88c0dc..9577e7a8 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 @@ -33,7 +33,6 @@ import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.math.BigInteger; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -71,129 +70,50 @@ public class PeriodicShardSyncManagerTest { shardSyncTaskManagerProvider, true); } - @Test - public void testIfHashRangesAreNotMergedWhenNoOverlappingIntervalsGiven() { - List hashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "23")); - add(deserialize("24", "30")); - }}; - List sortAndMergedHashRanges = PeriodicShardSyncManager - .sortAndMergeOverlappingHashRanges(hashRanges); - Assert.assertEquals(hashRanges, sortAndMergedHashRanges); - } - - @Test - public void testIfHashRangesAreSortedWhenNoOverlappingIntervalsGiven() { - List hashRanges = new ArrayList() {{ - add(deserialize("2", "3")); - add(deserialize("0", "1")); - add(deserialize("24", "30")); - add(deserialize("4", "23")); - }}; - List hashRangesCopy = new ArrayList<>(); - hashRangesCopy.addAll(hashRanges); - List sortAndMergedHashRanges = PeriodicShardSyncManager - .sortAndMergeOverlappingHashRanges(hashRangesCopy); - Assert.assertEquals(hashRangesCopy, sortAndMergedHashRanges); - Assert.assertNotEquals(hashRanges, sortAndMergedHashRanges); - } - - @Test - public void testIfHashRangesAreMergedWhenOverlappingIntervalsGivenCase1() { - List hashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "23")); - add(deserialize("6", "23")); - add(deserialize("24", "30")); - }}; - List expectedHashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "23")); - add(deserialize("24", "30")); - }}; - List sortAndMergedHashRanges = PeriodicShardSyncManager - .sortAndMergeOverlappingHashRanges(hashRanges); - Assert.assertEquals(expectedHashRanges, sortAndMergedHashRanges); - } - - @Test - public void testIfHashRangesAreMergedWhenOverlappingIntervalsGivenCase2() { - List hashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "5")); - add(deserialize("4", "23")); - add(deserialize("24", "30")); - }}; - List expectedHashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "23")); - add(deserialize("24", "30")); - }}; - List sortAndMergedHashRanges = PeriodicShardSyncManager - .sortAndMergeOverlappingHashRanges(hashRanges); - Assert.assertEquals(expectedHashRanges, sortAndMergedHashRanges); - } - - @Test - public void testIfHashRangesAreMergedWhenOverlappingIntervalsGivenCase3() { - List hashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "23")); - add(deserialize("4", "5")); - add(deserialize("24", "30")); - }}; - List expectedHashRanges = new ArrayList() {{ - add(deserialize("0", "1")); - add(deserialize("2", "3")); - add(deserialize("4", "23")); - add(deserialize("24", "30")); - }}; - List sortAndMergedHashRanges = PeriodicShardSyncManager - .sortAndMergeOverlappingHashRanges(hashRanges); - Assert.assertEquals(expectedHashRanges, sortAndMergedHashRanges); - } - @Test public void testForFailureWhenHashRangesAreIncomplete() { - List hashRanges = new ArrayList() {{ + List hashRanges = new ArrayList() {{ add(deserialize("0", "1")); add(deserialize("2", "3")); add(deserialize("4", "23")); add(deserialize("6", "23")); - add(deserialize("25", "30")); // Missing interval here - }}; + add(deserialize("25", MAX_HASH_KEY.toString())); // Missing interval here + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); Assert.assertTrue(PeriodicShardSyncManager - .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); } @Test public void testForSuccessWhenHashRangesAreComplete() { - List hashRanges = new ArrayList() {{ + List hashRanges = new ArrayList() {{ add(deserialize("0", "1")); add(deserialize("2", "3")); add(deserialize("4", "23")); add(deserialize("6", "23")); - add(deserialize("24", "30")); - }}; + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); Assert.assertFalse(PeriodicShardSyncManager - .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges, BigInteger.ZERO, BigInteger.valueOf(30)).isPresent()); + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); } @Test public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() { - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, null)); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, null).shouldDoShardSync()); } @Test public void testIfShardSyncIsInitiatedWhenEmptyLeasesArePassed() { - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, new ArrayList<>())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, new ArrayList<>()).shouldDoShardSync()); } @Test @@ -211,7 +131,7 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); } @Test @@ -229,8 +149,8 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @Test @@ -252,8 +172,8 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @Test @@ -271,8 +191,8 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @Test @@ -290,7 +210,7 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); List multiStreamLeases2 = new ArrayList() {{ add(deserialize(MIN_HASH_KEY.toString(), "1")); add(deserialize("2", "3")); // Hole between 3 and 5 @@ -305,8 +225,8 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); // Resetting the holes IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases2))); - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases2)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync()); } @Test @@ -324,7 +244,7 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); List multiStreamLeases2 = new ArrayList() {{ add(deserialize(MIN_HASH_KEY.toString(), "1")); add(deserialize("2", "3")); // Hole between 3 and 5 @@ -339,11 +259,11 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); // Resetting the holes IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases2))); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync())); // Resetting the holes IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @Test @@ -385,8 +305,8 @@ public class PeriodicShardSyncManagerTest { // Assert that shard sync should never trigger IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); - Assert.assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); // Assert that all the leases now has hashRanges set. for(Lease lease : multiStreamLeases) { @@ -433,8 +353,8 @@ public class PeriodicShardSyncManagerTest { // Assert that shard sync should never trigger IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert - .assertFalse(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases))); - Assert.assertTrue(periodicShardSyncManager.shouldDoShardSync(streamIdentifier, multiStreamLeases)); + .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); // Assert that all the leases now has hashRanges set. for(Lease lease : multiStreamLeases) { From 72a6d5e08496c2b56795039381ac609bab07bf85 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 25 May 2020 13:25:43 -0700 Subject: [PATCH 094/159] Adding more unit test cases and fixing an edge case --- .../kinesis/common/HashKeyRangeForLease.java | 7 + .../coordinator/PeriodicShardSyncManager.java | 13 +- .../amazon/kinesis/coordinator/Scheduler.java | 2 - .../PeriodicShardSyncManagerTest.java | 225 +++++++++++++++++- 4 files changed, 240 insertions(+), 7 deletions(-) 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 063451a0..d2540073 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 @@ -32,6 +32,13 @@ public class HashKeyRangeForLease { private final BigInteger startingHashKey; private final BigInteger endingHashKey; + public HashKeyRangeForLease(BigInteger startingHashKey, BigInteger endingHashKey) { + Validate.isTrue(startingHashKey.compareTo(endingHashKey) < 0, + "StartingHashKey %s must be less than EndingHashKey %s ", startingHashKey, endingHashKey); + this.startingHashKey = startingHashKey; + this.endingHashKey = endingHashKey; + } + /** * Serialize the startingHashKey for persisting in external storage * 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 a96bf01e..5ac4647c 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 @@ -15,6 +15,7 @@ package software.amazon.kinesis.coordinator; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ComparisonChain; import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.NonNull; @@ -245,7 +246,8 @@ class PeriodicShardSyncManager { private final String reasonForDecision; } - private Optional hasHoleInLeases(StreamIdentifier streamIdentifier, List leases) { + @VisibleForTesting + Optional hasHoleInLeases(StreamIdentifier streamIdentifier, List leases) { // Filter the leases with any checkpoint other than shard end. List activeLeases = leases.stream() .filter(lease -> lease.checkpoint() != null && !lease.checkpoint().isShardEnd()).collect(Collectors.toList()); @@ -385,13 +387,16 @@ class PeriodicShardSyncManager { private static final long serialVersionUID = 1L; - @Override public int compare(Lease lease, Lease otherLease) { + @Override + public int compare(Lease lease, Lease otherLease) { Validate.notNull(lease); Validate.notNull(otherLease); Validate.notNull(lease.hashKeyRangeForLease()); Validate.notNull(otherLease.hashKeyRangeForLease()); - return lease.hashKeyRangeForLease().startingHashKey() - .compareTo(otherLease.hashKeyRangeForLease().startingHashKey()); + return ComparisonChain.start() + .compare(lease.hashKeyRangeForLease().startingHashKey(), otherLease.hashKeyRangeForLease().startingHashKey()) + .compare(lease.hashKeyRangeForLease().endingHashKey(), otherLease.hashKeyRangeForLease().endingHashKey()) + .result(); } } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 91125a06..e2f2f852 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 @@ -352,8 +352,6 @@ public class Scheduler implements Runnable { } log.info("Scheduling periodicShardSync"); leaderElectedPeriodicShardSyncManager.start(); - // TODO: enable periodicShardSync after https://github.com/jushkem/amazon-kinesis-client/pull/2 is merged - // TODO: Determine if waitUntilHashRangeCovered() is needed. streamSyncWatch.start(); isDone = true; } catch (LeasingException e) { 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 9577e7a8..dfba2791 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 @@ -15,6 +15,8 @@ package software.amazon.kinesis.coordinator; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -23,6 +25,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.Shard; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.common.HashKeyRangeForLease; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; @@ -33,7 +36,9 @@ import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardSyncTaskManager; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import java.math.BigInteger; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -106,6 +111,44 @@ public class PeriodicShardSyncManagerTest { .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); } + @Test + public void testForSuccessWhenUnSortedHashRangesAreComplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("4", "23")); + add(deserialize("2", "3")); + add(deserialize("0", "1")); + add(deserialize("24", MAX_HASH_KEY.toString())); + add(deserialize("6", "23")); + + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenHashRangesAreCompleteForOverlappingLeasesAtEnd() { + List hashRanges = new ArrayList() {{ + add(deserialize("0", "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + add(deserialize("24", "45")); + }}.stream().map(hashKeyRangeForLease -> { + Lease lease = new MultiStreamLease(); + lease.hashKeyRange(hashKeyRangeForLease); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(streamIdentifier, hashRanges).isPresent()); + } + @Test public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() { Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, null).shouldDoShardSync()); @@ -135,7 +178,7 @@ public class PeriodicShardSyncManagerTest { } @Test - public void testIfShardSyncIsNotInitiatedWhenConfidenceFactorIsReached() { + public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsReached() { List multiStreamLeases = new ArrayList() {{ add(deserialize(MIN_HASH_KEY.toString(), "1")); add(deserialize("2", "3")); @@ -362,4 +405,184 @@ public class PeriodicShardSyncManagerTest { } } + @Test + public void testFor1000DifferentValidSplitHierarchyTreeTheHashRangesAreAlwaysComplete() { + for(int i=0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.SPLIT, maxInitialLeaseCount, false); + Collections.shuffle(leases); +// System.out.println( +// leases.stream().map(l -> l.checkpoint().sequenceNumber() + ":" + l.hashKeyRangeForLease()).collect(Collectors.toList())); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidMergeHierarchyTreeTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, false); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidReshardHierarchyTreeTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, false); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidMergeHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.MERGE, maxInitialLeaseCount, true); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + @Test + public void testFor1000DifferentValidReshardHierarchyTreeWithSomeInProgressParentsTheHashRangesAreAlwaysComplete() { + for (int i = 0; i < 1000; i++) { + int maxInitialLeaseCount = 100; + List leases = generateInitialLeases(maxInitialLeaseCount); + reshard(leases, 5, ReshardType.ANY, maxInitialLeaseCount, true); + Collections.shuffle(leases); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(streamIdentifier, leases).isPresent()); + } + } + + + + private List generateInitialLeases(int initialShardCount) { + long hashRangeInternalMax = 10000000; + List initialLeases = new ArrayList<>(); + long leaseStartKey = 0; + for (int i = 1; i <= initialShardCount; i++) { + final Lease lease = new Lease(); + long leaseEndKey; + if (i != initialShardCount) { + leaseEndKey = (hashRangeInternalMax / initialShardCount) * i; + lease.hashKeyRange(HashKeyRangeForLease.deserialize(leaseStartKey + "", leaseEndKey + "")); + } else { + leaseEndKey = 0; + lease.hashKeyRange(HashKeyRangeForLease.deserialize(leaseStartKey + "", MAX_HASH_KEY.toString())); + } + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + lease.leaseKey("shard-" + i); + initialLeases.add(lease); + leaseStartKey = leaseEndKey + 1; + } + return initialLeases; + } + + private void reshard(List initialLeases, int depth, ReshardType reshardType, int leaseCounter, + boolean shouldKeepSomeParentsInProgress) { + for (int i = 0; i < depth; i++) { + if (reshardType == ReshardType.SPLIT) { + leaseCounter = split(initialLeases, leaseCounter); + } else if (reshardType == ReshardType.MERGE) { + leaseCounter = merge(initialLeases, leaseCounter, shouldKeepSomeParentsInProgress); + } else { + if (isHeads()) { + leaseCounter = split(initialLeases, leaseCounter); + } else { + leaseCounter = merge(initialLeases, leaseCounter, shouldKeepSomeParentsInProgress); + } + } + } + } + + private int merge(List initialLeases, int leaseCounter, boolean shouldKeepSomeParentsInProgress) { + List leasesEligibleForMerge = initialLeases.stream().filter(l -> CollectionUtils.isNullOrEmpty(l.childShardIds())) + .collect(Collectors.toList()); +// System.out.println("Leases to merge : " + leasesEligibleForMerge); + int leasesToMerge = (int) ((leasesEligibleForMerge.size() - 1) / 2.0 * Math.random()); + for (int i = 0; i < leasesToMerge; i += 2) { + Lease parent1 = leasesEligibleForMerge.get(i); + Lease parent2 = leasesEligibleForMerge.get(i + 1); + if(parent2.hashKeyRangeForLease().startingHashKey().subtract(parent1.hashKeyRangeForLease().endingHashKey()).equals(BigInteger.ONE)) + { + parent1.checkpoint(ExtendedSequenceNumber.SHARD_END); + if (!shouldKeepSomeParentsInProgress || (shouldKeepSomeParentsInProgress && isOneFromDiceRoll())) { +// System.out.println("Deciding to keep parent in progress : " + parent2); + parent2.checkpoint(ExtendedSequenceNumber.SHARD_END); + } + Lease child = new Lease(); + child.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child.leaseKey("shard-" + (++leaseCounter)); +// System.out.println("Parent " + parent1 + " and " + parent2 + " merges into " + child); + child.hashKeyRange(new HashKeyRangeForLease(parent1.hashKeyRangeForLease().startingHashKey(), + parent2.hashKeyRangeForLease().endingHashKey())); + parent1.childShardIds(Collections.singletonList(child.leaseKey())); + parent2.childShardIds(Collections.singletonList(child.leaseKey())); + child.parentShardIds(Sets.newHashSet(parent1.leaseKey(), parent2.leaseKey())); + + initialLeases.add(child); + } + } + return leaseCounter; + } + + private int split(List initialLeases, int leaseCounter) { + List leasesEligibleForSplit = initialLeases.stream().filter(l -> CollectionUtils.isNullOrEmpty(l.childShardIds())) + .collect(Collectors.toList()); +// System.out.println("Leases to split : " + leasesEligibleForSplit); + int leasesToSplit = (int) (leasesEligibleForSplit.size() * Math.random()); + for (int i = 0; i < leasesToSplit; i++) { + Lease parent = leasesEligibleForSplit.get(i); + parent.checkpoint(ExtendedSequenceNumber.SHARD_END); + Lease child1 = new Lease(); + child1.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child1.hashKeyRange(new HashKeyRangeForLease(parent.hashKeyRangeForLease().startingHashKey(), + parent.hashKeyRangeForLease().startingHashKey().add(parent.hashKeyRangeForLease().endingHashKey()) + .divide(new BigInteger("2")))); + child1.leaseKey("shard-" + (++leaseCounter)); + Lease child2 = new Lease(); + child2.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child2.hashKeyRange(new HashKeyRangeForLease( + parent.hashKeyRangeForLease().startingHashKey().add(parent.hashKeyRangeForLease().endingHashKey()) + .divide(new BigInteger("2")).add(new BigInteger("1")), + parent.hashKeyRangeForLease().endingHashKey())); + child2.leaseKey("shard-" + (++leaseCounter)); + + child1.parentShardIds(Sets.newHashSet(parent.leaseKey())); + child2.parentShardIds(Sets.newHashSet(parent.leaseKey())); + parent.childShardIds(Lists.newArrayList(child1.leaseKey(), child2.leaseKey())); + +// System.out.println("Parent " + parent + " splits into " + child1 + " and " + child2); + + initialLeases.add(child1); + initialLeases.add(child2); + } + return leaseCounter; + } + + private boolean isHeads() { + return Math.random() <= 0.5; + } + + private boolean isOneFromDiceRoll() { + return Math.random() <= 0.16; + } + + + private enum ReshardType { + SPLIT, + MERGE, + ANY + } + + + } From 08ca1b61bc596858c0596ec7644c5c3ef482564e Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 25 May 2020 13:37:10 -0700 Subject: [PATCH 095/159] Changing test case name --- .../amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 a9e9c689..06a72230 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/BlockOnParentShardTaskTest.java @@ -75,7 +75,7 @@ public class BlockOnParentShardTaskTest { * @throws DependencyException */ @Test - public final void testCallWhenParentsHaveFinished() + public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinished() throws DependencyException, InvalidStateException, ProvisionedThroughputException { ShardInfo shardInfo = null; @@ -116,7 +116,7 @@ public class BlockOnParentShardTaskTest { * @throws DependencyException */ @Test - public final void testCallWhenParentsHaveFinishedMultiStream() + public final void testCallShouldNotThrowBlockedOnParentWhenParentsHaveFinishedMultiStream() throws DependencyException, InvalidStateException, ProvisionedThroughputException { ShardInfo shardInfo = null; From 67d2b082fd7580e01cb302cf2d73bc3441f19809 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 26 May 2020 11:10:10 -0700 Subject: [PATCH 096/159] Fixing retry logic --- .../coordinator/PeriodicShardSyncManager.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 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 5ac4647c..c84547e2 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 @@ -47,6 +47,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.Executors; @@ -166,7 +167,7 @@ class PeriodicShardSyncManager { final ShardSyncResponse shardSyncResponse = checkForShardSync(streamConfigEntry.getKey(), streamToLeasesMap.get(streamConfigEntry.getKey())); if (shardSyncResponse.shouldDoShardSync()) { - log.info("Periodic shard syncer initiating shard sync for {} due to the reason - ", + log.info("Periodic shard syncer initiating shard sync for {} due to the reason - {} ", streamConfigEntry.getKey(), shardSyncResponse.reasonForDecision()); final ShardSyncTaskManager shardSyncTaskManager = shardSyncTaskManagerProvider .apply(streamConfigEntry.getValue()); @@ -228,7 +229,7 @@ class PeriodicShardSyncManager { .hasHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); return new ShardSyncResponse(hasHoleWithHighConfidence, "Detected same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() - + " times. Shard sync will be initiated when threshold breaches " + + " times. Shard sync will be initiated when threshold reaches " + CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY); } else { @@ -308,8 +309,8 @@ class PeriodicShardSyncManager { log.error("Incomplete hash range found for stream {} between {} and {}.", streamIdentifier, sortedLeasesWithHashKeyRanges.get(0), sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1)); - return Optional.of(new HashRangeHole(sortedLeasesWithHashKeyRanges.get(0), - sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1))); + return Optional.of(new HashRangeHole(sortedLeasesWithHashKeyRanges.get(0).hashKeyRangeForLease(), + sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1).hashKeyRangeForLease())); } // Check for any holes in the sorted hashrange intervals. if (sortedLeasesWithHashKeyRanges.size() > 1) { @@ -330,8 +331,8 @@ class PeriodicShardSyncManager { if (!rangeDiff.equals(BigInteger.ONE)) { log.error("Incomplete hash range found for {} between {} and {}.", streamIdentifier, leftMostLeaseToReportInCaseOfHole, sortedLeasesWithHashKeyRanges.get(i)); - return Optional.of(new HashRangeHole(leftMostLeaseToReportInCaseOfHole, - sortedLeasesWithHashKeyRanges.get(i))); + return Optional.of(new HashRangeHole(leftMostLeaseToReportInCaseOfHole.hashKeyRangeForLease(), + sortedLeasesWithHashKeyRanges.get(i).hashKeyRangeForLease())); } leftMostLeaseToReportInCaseOfHole = sortedLeasesWithHashKeyRanges.get(i); leftLeaseHashRange = rightLeaseHashRange; @@ -352,16 +353,16 @@ class PeriodicShardSyncManager { @Value private static class HashRangeHole { HashRangeHole() { - leaseAtEndOfPossibleHole = leaseAtStartOfPossibleHole = null; + hashRangeAtStartOfPossibleHole = hashRangeAtEndOfPossibleHole = null; } - HashRangeHole(Lease leaseAtStartOfPossibleHole, Lease leaseAtEndOfPossibleHole) { - this.leaseAtStartOfPossibleHole = leaseAtStartOfPossibleHole; - this.leaseAtEndOfPossibleHole = leaseAtEndOfPossibleHole; + HashRangeHole(HashKeyRangeForLease hashRangeAtStartOfPossibleHole, HashKeyRangeForLease hashRangeAtEndOfPossibleHole) { + this.hashRangeAtStartOfPossibleHole = hashRangeAtStartOfPossibleHole; + this.hashRangeAtEndOfPossibleHole = hashRangeAtEndOfPossibleHole; } - private final Lease leaseAtStartOfPossibleHole; - private final Lease leaseAtEndOfPossibleHole; + private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole; + private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; } private static class HashRangeHoleTracker { From 113029e33cb9366d2dc2883f47c38e9241744232 Mon Sep 17 00:00:00 2001 From: Micah Jaffe <31011877+micah-jaffe@users.noreply.github.com> Date: Sun, 31 May 2020 10:28:05 -0700 Subject: [PATCH 097/159] Update lease sync algorithm to create leases for no more than one level of the shard hierarchy (#41) * Update NonEmptyLeaseTableSynchronizer algorithm to create leases for no more than one level of the shard hierarchy * Add and fix unit tests for LATEST, TRIM_HORIZON, AT_TIMESTAMP * Update unit tests * Fix edge case of reading from TRIM with partial lease for one child shard * Revert child shards optimization for TRIM/TIMESTAMP case --- .../leases/HierarchicalShardSyncer.java | 202 ++- .../leases/HierarchicalShardSyncerTest.java | 1413 +++++++++++++---- 2 files changed, 1239 insertions(+), 376 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 2b2df48c..a2700097 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 @@ -18,9 +18,11 @@ import java.io.Serializable; import java.math.BigInteger; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -33,6 +35,7 @@ import java.util.stream.Collectors; import com.google.common.annotations.VisibleForTesting; import lombok.AllArgsConstructor; import lombok.Data; +import lombok.NoArgsConstructor; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; @@ -446,7 +449,7 @@ public class HierarchicalShardSyncer { /** * Note: Package level access for testing purposes only. * Check if this shard is a descendant of a shard that is (or will be) processed. - * Create leases for the ancestors of this shard as required. + * Create leases for the first ancestor of this shard that needs to be processed, as required. * See javadoc of determineNewLeasesToCreate() for rules and example. * * @param shardId The shardId to check. @@ -462,10 +465,10 @@ public class HierarchicalShardSyncer { static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, final Map shardIdToShardMapOfAllKinesisShards, - final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext, + final Map shardIdToLeaseMapOfNewShards, final MemoizationContext memoizationContext, final MultiStreamArgs multiStreamArgs) { final String streamIdentifier = getStreamIdentifier(multiStreamArgs); - final Boolean previousValue = memoizationContext.get(shardId); + final Boolean previousValue = memoizationContext.isDescendant(shardId); if (previousValue != null) { return previousValue; } @@ -480,13 +483,17 @@ public class HierarchicalShardSyncer { // We don't need to add leases of its ancestors, // because we'd have done it when creating a lease for this shard. } else { + final Shard shard = shardIdToShardMapOfAllKinesisShards.get(shardId); final Set parentShardIds = getParentShardIds(shard, shardIdToShardMapOfAllKinesisShards); for (String parentShardId : parentShardIds) { - // Check if the parent is a descendant, and include its ancestors. - if (checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, initialPosition, - shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, - memoizationContext, multiStreamArgs)) { + // Check if the parent is a descendant, and include its ancestors. Or, if the parent is NOT a + // descendant but we should create a lease for it anyway (e.g. to include in processing from + // TRIM_HORIZON or AT_TIMESTAMP). If either is true, then we mark the current shard as a descendant. + final boolean isParentDescendant = checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, + initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, + shardIdToLeaseMapOfNewShards, memoizationContext, multiStreamArgs); + if (isParentDescendant || memoizationContext.shouldCreateLease(parentShardId)) { isDescendant = true; descendantParentShardIds.add(parentShardId); log.debug("{} : Parent shard {} is a descendant.", streamIdentifier, parentShardId); @@ -499,48 +506,87 @@ public class HierarchicalShardSyncer { if (isDescendant) { for (String parentShardId : parentShardIds) { if (!shardIdsOfCurrentLeases.contains(parentShardId)) { - log.debug("{} : Need to create a lease for shardId {}", streamIdentifier, parentShardId); Lease lease = shardIdToLeaseMapOfNewShards.get(parentShardId); + + /** + * If the lease for the parent shard does not already exist, there are two cases in which we + * would want to create it: + * - If we have already marked the parentShardId for lease creation in a prior recursive + * call. This could happen if we are trying to process from TRIM_HORIZON or AT_TIMESTAMP. + * - If the parent shard is not a descendant but the current shard is a descendant, then + * the parent shard is the oldest shard in the shard hierarchy that does not have an + * ancestor in the lease table (the adjacent parent is necessarily a descendant, and + * therefore covered in the lease table). So we should create a lease for the parent. + */ if (lease == null) { - lease = multiStreamArgs.isMultiStreamMode() ? - newKCLMultiStreamLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId), - multiStreamArgs.streamIdentifier()) : - newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); - shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + if (memoizationContext.shouldCreateLease(parentShardId) || + !descendantParentShardIds.contains(parentShardId)) { + log.debug("{} : Need to create a lease for shardId {}", streamIdentifier, parentShardId); + lease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId), + multiStreamArgs.streamIdentifier()) : + newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); + shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + } } - if (descendantParentShardIds.contains(parentShardId) - && !initialPosition.getInitialPositionInStream() + /** + * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the + * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will + * add a lease just like we do for TRIM_HORIZON. However we will only return back records + * with server-side timestamp at or after the specified initial position timestamp. + * + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: (4, 5, 7) + * + * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with + * timestamp value 206. We will then create new leases for all the shards 0 and 1 (with + * checkpoint set AT_TIMESTAMP), even though these ancestor shards have an epoch less than + * 206. However as we begin processing the ancestor shards, their checkpoints would be + * updated to SHARD_END and their leases would then be deleted since they won't have records + * with server-side timestamp at/after 206. And after that we will begin processing the + * descendant shards with epoch at/after 206 and we will return the records that meet the + * timestamp requirement for these shards. + */ + if (lease != null) { + if (descendantParentShardIds.contains(parentShardId) + && !initialPosition.getInitialPositionInStream() .equals(InitialPositionInStream.AT_TIMESTAMP)) { - lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - lease.checkpoint(convertToCheckpoint(initialPosition)); + lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } else { + lease.checkpoint(convertToCheckpoint(initialPosition)); + } } } } } else { - // This shard should be included, if the customer wants to process all records in the stream or - // if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a lease just like we do - // for TRIM_HORIZON. However we will only return back records with server-side timestamp at or - // after the specified initial position timestamp. + // This shard is not a descendant, but should still be included if the customer wants to process all + // records in the stream or if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a + // lease just like we do for TRIM_HORIZON. However we will only return back records with server-side + // timestamp at or after the specified initial position timestamp. if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON) || initialPosition.getInitialPositionInStream() .equals(InitialPositionInStream.AT_TIMESTAMP)) { - isDescendant = true; + memoizationContext.setShouldCreateLease(shardId, true); } } - } } - memoizationContext.put(shardId, isDescendant); + memoizationContext.setIsDescendant(shardId, isDescendant); return isDescendant; } static boolean checkIfDescendantAndAddNewLeasesForAncestors(final String shardId, final InitialPositionInStreamExtended initialPosition, final Set shardIdsOfCurrentLeases, final Map shardIdToShardMapOfAllKinesisShards, - final Map shardIdToLeaseMapOfNewShards, final Map memoizationContext) { + final Map shardIdToLeaseMapOfNewShards, MemoizationContext memoizationContext) { return checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, memoizationContext, new MultiStreamArgs(false, null)); @@ -1033,8 +1079,10 @@ public class HierarchicalShardSyncer { * Note: Package level access only for testing purposes. *

* For each open (no ending sequence number) shard without open parents that doesn't already have a lease, - * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): - * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. + * determine if it is a descendant of any shard which is or will be processed (e.g. for which a lease exists): + * If so, create a lease for the first ancestor that needs to be processed (if needed). We will create leases + * for no more than one level in the ancestry tree. Once we find the first ancestor that needs to be processed, + * we will avoid creating leases for further descendants of that ancestor. * If not, set checkpoint of the shard to the initial position specified by the client. * To check if we need to create leases for ancestors, we use the following rules: * * If we began (or will begin) processing data for a shard, then we must reach end of that shard before @@ -1052,11 +1100,19 @@ public class HierarchicalShardSyncer { * Shard structure (each level depicts a stream segment): * 0 1 2 3 4 5 - shards till epoch 102 * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | / \ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) - * New leases to create: (2, 6, 7, 8, 9, 10) + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | / \ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: (4, 5, 7) + * + * If initial position is LATEST: + * - New leases to create: (6) + * If initial position is TRIM_HORIZON: + * - New leases to create: (0, 1) + * If initial position is AT_TIMESTAMP(epoch=200): + * - New leases to create: (0, 1) + * *

* The leases returned are sorted by the starting sequence number - following the same order * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail @@ -1083,7 +1139,7 @@ public class HierarchicalShardSyncer { .collect(Collectors.toSet()); final List openShards = getOpenShards(shards, streamIdentifier); - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); // Iterate over the open shards and find those that don't have any lease entries. for (Shard shard : openShards) { @@ -1094,45 +1150,32 @@ public class HierarchicalShardSyncer { } else if (inconsistentShardIds.contains(shardId)) { log.info("{} : shardId {} is an inconsistent child. Not creating a lease", streamIdentifier, shardId); } else { - log.debug("{} : Need to create a lease for shardId {}", streamIdentifier, shardId); - final Lease newLease = multiStreamArgs.isMultiStreamMode() ? - newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : - newKCLLease(shard); + log.debug("{} : Beginning traversal of ancestry tree for shardId {}", streamIdentifier, shardId); + + // A shard is a descendant if at least one if its ancestors exists in the lease table. + // We will create leases for only one level in the ancestry tree. Once we find the first ancestor + // that needs to be processed in order to complete the hash range, we will not create leases for + // further descendants of that ancestor. final boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, memoizationContext, multiStreamArgs); - /** - * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the - * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a - * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side - * timestamp at or after the specified initial position timestamp. - * - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * - * Current leases: empty set - * - * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with - * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to - * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin - * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases - * would then be deleted since they won't have records with server-side timestamp at/after 206. And - * after that we will begin processing the descendant shards with epoch at/after 206 and we will - * return the records that meet the timestamp requirement for these shards. - */ - if (isDescendant - && !initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { - newLease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { + // If shard is a descendant, the leases for its ancestors were already created above. Open shards + // that are NOT descendants will not have leases yet, so we create them here. We will not create + // leases for open shards that ARE descendants yet - leases for these shards will be created upon + // SHARD_END of their parents. + if (!isDescendant) { + log.debug("{} : shardId {} has no ancestors. Creating a lease.", streamIdentifier, shardId); + final Lease newLease = multiStreamArgs.isMultiStreamMode() ? + newKCLMultiStreamLease(shard, multiStreamArgs.streamIdentifier()) : + newKCLLease(shard); newLease.checkpoint(convertToCheckpoint(initialPosition)); + log.debug("{} : Set checkpoint of {} to {}", streamIdentifier, newLease.leaseKey(), newLease.checkpoint()); + shardIdToNewLeaseMap.put(shardId, newLease); + } else { + log.debug("{} : shardId {} is a descendant whose ancestors should already have leases. " + + "Not creating a lease.", streamIdentifier, shardId); } - log.debug("{} : Set checkpoint of {} to {}", streamIdentifier, newLease.leaseKey(), newLease.checkpoint()); - shardIdToNewLeaseMap.put(shardId, newLease); } } @@ -1143,4 +1186,29 @@ public class HierarchicalShardSyncer { return newLeasesToCreate; } } + + /** + * Helper class to pass around state between recursive traversals of shard hierarchy. + */ + @NoArgsConstructor + static class MemoizationContext { + private Map isDescendantMap = new HashMap<>(); + private Map shouldCreateLeaseMap = new HashMap<>(); + + Boolean isDescendant(String shardId) { + return isDescendantMap.get(shardId); + } + + void setIsDescendant(String shardId, Boolean isDescendant) { + isDescendantMap.put(shardId, isDescendant); + } + + Boolean shouldCreateLease(String shardId) { + return shouldCreateLeaseMap.computeIfAbsent(shardId, x -> Boolean.FALSE); + } + + void setShouldCreateLease(String shardId, Boolean shouldCreateLease) { + shouldCreateLeaseMap.put(shardId, shouldCreateLease); + } + } } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java index acfa3c51..096bf33a 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 @@ -20,7 +20,6 @@ package software.amazon.kinesis.leases; import static org.hamcrest.CoreMatchers.equalTo; 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; @@ -34,6 +33,7 @@ import java.math.BigInteger; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -43,7 +43,6 @@ import java.util.Set; import java.util.UUID; import java.util.function.Consumer; import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; @@ -71,6 +70,8 @@ import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.NullMetricsScope; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import static software.amazon.kinesis.leases.HierarchicalShardSyncer.MemoizationContext; + @RunWith(MockitoJUnitRunner.class) // CHECKSTYLE:IGNORE JavaNCSS FOR NEXT 800 LINES public class HierarchicalShardSyncerTest { @@ -473,14 +474,180 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: TRIM_HORIZON + * Leases to create: (0, 1, 2, 3, 4, 5) + */ @Test - public void testCheckAndCreateLeasesForNewShardsAtTrimHorizon() throws Exception { - testCheckAndCreateLeaseForShardsIfMissing(constructShardListForGraphA(), INITIAL_POSITION_TRIM_HORIZON); + public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithEmptyLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", + "shardId-3", "shardId-4", "shardId-5")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); } + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: AT_TIMESTAMP(1000) + * Leases to create: (8, 4, 9, 10) + */ @Test - public void testCheckAndCreateLeasesForNewShardsAtTimestamp() throws Exception { - testCheckAndCreateLeaseForShardsIfMissing(constructShardListForGraphA(), INITIAL_POSITION_AT_TIMESTAMP); + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable1() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", + "shardId-10")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: AT_TIMESTAMP(200) + * Leases to create: (6, 7, 4, 5) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-6", "shardId-7", "shardId-4", + "shardId-5")); + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPositionAtTimestamp(new Date(200L)); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: LATEST + * Leases to create: (8, 4, 9, 10) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", + "shardId-10")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: TRIM_HORIZON + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithPartialLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from TRIM_HORIZON. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: AT_TIMESTAMP(1000) + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable1() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from AT_TIMESTAMP. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: AT_TIMESTAMP(200) + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPositionAtTimestamp(new Date(200L)); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from AT_TIMESTAMP. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: LATEST + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from LATEST. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.shardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate, existingLeases); } @Test(expected = KinesisClientLibIOException.class) @@ -657,7 +824,7 @@ public class HierarchicalShardSyncerTest { final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); @@ -703,7 +870,8 @@ public class HierarchicalShardSyncerTest { } private void testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions( - final ExtendedSequenceNumber sequenceNumber, final InitialPositionInStreamExtended position) + final ExtendedSequenceNumber sequenceNumber, + final InitialPositionInStreamExtended position) throws Exception { final String shardIdPrefix = "shardId-%d"; final List shards = constructShardListForGraphA(); @@ -732,7 +900,7 @@ public class HierarchicalShardSyncerTest { final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); @@ -843,7 +1011,7 @@ public class HierarchicalShardSyncerTest { SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); @@ -933,7 +1101,8 @@ public class HierarchicalShardSyncerTest { SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - final Set expectedCreateLeases = new HashSet<>(createLeasesFromShards(shards, sequenceNumber, null)); + + final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); assertThat(createLeases, equalTo(expectedCreateLeases)); verify(shardDetector, times(2)).listShards(); @@ -1079,19 +1248,33 @@ public class HierarchicalShardSyncerTest { throws Exception { final String shardId0 = "shardId-0"; final String shardId1 = "shardId-1"; - final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); - final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange), - ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); + final HashKeyRange range1 = ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, BigInteger.ONE.toString()); + final HashKeyRange range2 = ShardObjectHelper.newHashKeyRange(new BigInteger("2").toString(), ShardObjectHelper.MAX_HASH_KEY); + final SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("11", null); + final List shards = Arrays.asList(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange, range1), + ShardObjectHelper.newShard(shardId1, null, null, sequenceRange, range2)); + final Set expectedLeaseKeys = new HashSet<>(Arrays.asList(shardId0, shardId1)); - testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeys); } private void testCheckAndCreateLeaseForShardsIfMissing(final List shards, - final InitialPositionInStreamExtended initialPosition) throws Exception { + final InitialPositionInStreamExtended initialPosition, + final Set expectedLeaseKeys) throws Exception { + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeys, Collections.emptyList()); + } + + private void testCheckAndCreateLeaseForShardsIfMissing(final List shards, + final InitialPositionInStreamExtended initialPosition, + final Set expectedLeaseKeys, + final List existingLeases) throws Exception { + final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()); + when(shardDetector.listShardsWithFilter(any())).thenReturn(getFilteredShards(shards, initialPosition)); + when(dynamoDBLeaseRefresher.listLeases()).thenReturn(existingLeases); + when(dynamoDBLeaseRefresher.isLeaseTableEmpty()).thenReturn(existingLeases.isEmpty()); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCaptor.capture())).thenReturn(true); hierarchicalShardSyncer @@ -1102,16 +1285,15 @@ public class HierarchicalShardSyncerTest { final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set leaseSequenceNumbers = leases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - final Set expectedLeaseKeys = shards.stream().map(Shard::shardId).collect(Collectors.toSet()); + final Set expectedSequenceNumbers = new HashSet<>(Collections .singletonList(new ExtendedSequenceNumber(initialPosition.getInitialPositionInStream().name()))); - assertThat(leases.size(), equalTo(shards.size())); + assertThat(leases.size(), equalTo(expectedLeaseKeys.size())); assertThat(leaseKeys, equalTo(expectedLeaseKeys)); assertThat(leaseSequenceNumbers, equalTo(expectedSequenceNumbers)); - verify(shardDetector).listShards(); - verify(dynamoDBLeaseRefresher, times(shards.size())).createLeaseIfNotExists(any(Lease.class)); + verify(dynamoDBLeaseRefresher, times(expectedLeaseKeys.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } @@ -1177,49 +1359,30 @@ public class HierarchicalShardSyncerTest { assertThat(newLeases.get(0).leaseKey(), equalTo(lastShardId)); } -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) -// * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) -// * Current leases: (3, 4, 5) -// */ + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (3, 4, 5) + * Initial position: LATEST + * Expected leases: (2, 6) + */ @Test - public void testDetermineNewLeasesToCreateSplitMergeLatest1() { + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange1() { final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), - newLease("shardId-5")); - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_LATEST); - + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); final Map expectedShardIdCheckpointMap = new HashMap<>(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.LATEST); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); - - assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); - for (Lease lease : newLeases) { - assertThat("Unexpected lease: " + lease, expectedShardIdCheckpointMap.containsKey(lease.leaseKey()), - equalTo(true)); - assertThat(lease.checkpoint(), equalTo(expectedShardIdCheckpointMap.get(lease.leaseKey()))); - } + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) + * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): * 0 1 2 3 4 5- shards till epoch 102 * \ / \ / | | @@ -1227,12 +1390,796 @@ public class HierarchicalShardSyncerTest { * \ / | /\ * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (4, 5, 7) + * Initial position: LATEST + * Expected leases: (6) + * */ @Test - public void testDetermineNewLeasesToCreateSplitMergeLatest2() { + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange2() { final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), - newLease("shardId-7")); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: LATEST + * Expected leases: (3, 4, 9, 10) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: LATEST + * Expected leases: (8) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is + * epochs): 0 1 2 3 - shards till + * / \ | \ / + * 4 5 1 6 - shards from epoch 103 - 205 + * / \ / \ | | + * 7 8 9 10 1 6 + * shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (9, 10) + * Initial position: LATEST + * Expected leases: (1, 6, 7, 8) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestC_PartialHashRange5() { + final List shards = constructShardListForGraphC(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: LATEST + * Expected leases: (4, 8, 9, 10) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (6) + * Initial position: LATEST + * Expected leases: (7) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_PartialHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("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 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (5) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: empty set + * Initial position: LATEST + * Expected leases: (9, 10) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 7) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: TRIM_HORIZON + * Expected leases: (3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2, 3) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2, 3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (6) + * Initial position: TRIM_HORIZON + * Expected leases: (7) + */ +// TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases +// @Test +// public void testDetermineNewLeasesToCreateSplitMergeHorizonB_PartialHashRange() { +// final List shards = constructShardListForGraphB(); +// final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); +// final Map expectedShardIdCheckpointMap = new HashMap<>(); +// expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); +// assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); +// } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (5) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: empty set + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (3, 4, 5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2, 3) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2, 3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (6) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (7) + */ +// TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases +// @Test +// public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_PartialHashRange() { +// final List shards = constructShardListForGraphB(); +// final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); +// final Map expectedShardIdCheckpointMap = new HashMap<>(); +// expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.AT_TIMESTAMP); +// assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); +// } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /* + * Shard structure (x-axis is epochs): + * 0 3 6 9 + * \ / \ / \ / + * 2 5 8 + * / \ / \ / \ + * 1 4 7 10 + * + * Current leases: empty set + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + assertExpectedLeasesAreCreated(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + private void assertExpectedLeasesAreCreated(List shards, + List shardIdsOfCurrentLeases, + InitialPositionInStreamExtended initialPosition, + Map expectedShardIdCheckpointMap) { + + final List currentLeases = shardIdsOfCurrentLeases.stream() + .map(shardId -> newLease(shardId)).collect(Collectors.toList()); final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer @@ -1242,13 +2189,7 @@ public class HierarchicalShardSyncerTest { new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_LATEST); - - final Map expectedShardIdCheckpointMap = new HashMap<>(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + shards, currentLeases, initialPosition); assertThat(newLeases.size(), equalTo(expectedShardIdCheckpointMap.size())); for (Lease lease : newLeases) { @@ -1258,243 +2199,6 @@ public class HierarchicalShardSyncerTest { } } -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) -// * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) -// * Current leases: (3, 4, 5) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeHorizon1() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), - newLease("shardId-5")); - - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", - "shardId-6", "shardId-7", "shardId-8", "shardId-9", "shardId-10")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.TRIM_HORIZON)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) -// * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) -// * Current leases: (4, 5, 7) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeHorizon2() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), - newLease("shardId-7")); - - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>( - Arrays.asList("shardId-8", "shardId-9", "shardId-10", "shardId-6", "shardId-0", "shardId-1")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.TRIM_HORIZON)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) -// * For shard graph B (see the construct method doc for structure). -// * -// * Current leases: empty set -// */ - @Test - public void testDetermineNewLeasesToCreateGraphBNoInitialLeasesTrim() { - final List shards = constructShardListForGraphB(); - final List currentLeases = new ArrayList<>(); - - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.TRIM_HORIZON)); - final Set expectedLeaseKeys = IntStream.range(0, 11).mapToObj(id -> String.format("shardId-%d", id)) - .collect(Collectors.toSet()); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) -// * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) -// * Current leases: (3, 4, 5) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeAtTimestamp1() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-3"), newLease("shardId-4"), - newLease("shardId-5")); - - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", - "shardId-6", "shardId-7", "shardId-8", "shardId-9", "shardId-10")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.AT_TIMESTAMP)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - -// /** -// * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) -// * Shard structure (each level depicts a stream segment): -// * 0 1 2 3 4 5- shards till epoch 102 -// * \ / \ / | | -// * 6 7 4 5- shards from epoch 103 - 205 -// * \ / | /\ -// * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) -// * Current leases: (4, 5, 7) -// */ - @Test - public void testDetermineNewLeasesToCreateSplitMergeAtTimestamp2() { - final List shards = constructShardListForGraphA(); - final List currentLeases = Arrays.asList(newLease("shardId-4"), newLease("shardId-5"), - newLease("shardId-7")); - - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = new HashSet<>( - Arrays.asList("shardId-0", "shardId-1", "shardId-6", "shardId-8", "shardId-9", "shardId-10")); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.AT_TIMESTAMP)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - - /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) For shard graph B (see the - * construct method doc for structure). Current leases: empty set - */ - @Test - public void testDetermineNewLeasesToCreateGraphBNoInitialLeasesAtTimestamp() { - final List shards = constructShardListForGraphB(); - final List currentLeases = new ArrayList<>(); - - final Map shardIdToShardMap = HierarchicalShardSyncer.constructShardIdToShardMap(shards); - final Map> shardIdToChildShardIdsMap = HierarchicalShardSyncer - .constructShardIdToChildShardIdsMap(shardIdToShardMap); - - final HierarchicalShardSyncer.LeaseSynchronizer nonEmptyLeaseTableSynchronizer = - new HierarchicalShardSyncer.NonEmptyLeaseTableSynchronizer(shardDetector, shardIdToShardMap, shardIdToChildShardIdsMap); - - final List newLeases = HierarchicalShardSyncer.determineNewLeasesToCreate(nonEmptyLeaseTableSynchronizer, - shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - final Set leaseKeys = newLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final List checkpoints = newLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toList()); - final Set checkpoint = new HashSet<>(checkpoints); - - final Set expectedLeaseKeys = IntStream.range(0, shards.size()) - .mapToObj(id -> String.format("shardId-%d", id)).collect(Collectors.toSet()); - final Set expectedCheckpoint = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.AT_TIMESTAMP)); - - assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); - assertThat(checkpoints.size(), equalTo(expectedLeaseKeys.size())); - assertThat(leaseKeys, equalTo(expectedLeaseKeys)); - assertThat(checkpoint, equalTo(expectedCheckpoint)); - } - /* * 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 @@ -1536,6 +2240,57 @@ public class HierarchicalShardSyncerTest { ShardObjectHelper.newHashKeyRange("800", ShardObjectHelper.MAX_HASH_KEY))); } + /** + * Helper method to mimic behavior of Kinesis ListShardsWithFilter calls. + */ + private static List getFilteredShards(List shards, InitialPositionInStreamExtended initialPosition) { + switch (initialPosition.getInitialPositionInStream()) { + case LATEST: + return shards.stream() + .filter(s -> s.sequenceNumberRange().endingSequenceNumber() == null) + .collect(Collectors.toList()); + case TRIM_HORIZON: + String minSeqNum = shards.stream() + .min(Comparator.comparingLong(s -> Long.parseLong(s.sequenceNumberRange().startingSequenceNumber()))) + .map(s -> s.sequenceNumberRange().startingSequenceNumber()) + .orElseThrow(RuntimeException::new); + return shards.stream() + .filter(s -> s.sequenceNumberRange().startingSequenceNumber().equals(minSeqNum)) + .collect(Collectors.toList()); + case AT_TIMESTAMP: + return shards.stream() + .filter(s -> new Date(Long.parseLong(s.sequenceNumberRange().startingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) <= 0) + .filter(s -> s.sequenceNumberRange().endingSequenceNumber() == null || + new Date(Long.parseLong(s.sequenceNumberRange().endingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) > 0) + .collect(Collectors.toList()); + } + throw new RuntimeException("Unsupported initial position " + initialPosition); + } + + /* + * Helper method to get expected shards for Graph A based on initial position in stream. Shard structure (y-axis is + * epochs): 0 1 2 3 4 5- shards till + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - + * shards from epoch 206 (open - no ending sequenceNumber) + */ + private Set getExpectedLeasesForGraphA(List shards, + ExtendedSequenceNumber sequenceNumber, + InitialPositionInStreamExtended initialPosition) { + final List filteredShards; + if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { + // Lease creation for AT_TIMESTAMP should work the same as for TRIM_HORIZON - ignore shard filters + filteredShards = getFilteredShards(shards, INITIAL_POSITION_TRIM_HORIZON); + } else { + filteredShards = getFilteredShards(shards, initialPosition); + } + return new HashSet<>(createLeasesFromShards(filteredShards, sequenceNumber, null)); + } + // /* // * Helper method to construct a shard list for graph B. Graph B is defined below. // * Shard structure (x-axis is epochs): @@ -1571,12 +2326,53 @@ public class HierarchicalShardSyncerTest { ShardObjectHelper.newShard("shardId-10", null, "shardId-8", range6, hashRange1)); } + /** + * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is + * epochs): 0 1 2 3 - shards till + * / \ | \ / + * 4 5 1 6 - shards from epoch 103 - 205 + * / \ / \ | | + * 7 8 9 10 1 6 + * shards from epoch 206 (open - no ending sequenceNumber) + */ + private List constructShardListForGraphC() { + final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); + final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); + final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("103", null); + final SequenceNumberRange range3 = ShardObjectHelper.newSequenceNumberRange("103", "205"); + final SequenceNumberRange range4 = ShardObjectHelper.newSequenceNumberRange("206", null); + + return Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, range0, + ShardObjectHelper.newHashKeyRange("0", "399")), + ShardObjectHelper.newShard("shardId-1", null, null, range1, + ShardObjectHelper.newHashKeyRange("400", "499")), + ShardObjectHelper.newShard("shardId-2", null, null, range0, + ShardObjectHelper.newHashKeyRange("500", "599")), + ShardObjectHelper.newShard("shardId-3", null, null, range0, + ShardObjectHelper.newHashKeyRange("600", ShardObjectHelper.MAX_HASH_KEY)), + ShardObjectHelper.newShard("shardId-4", "shardId-0", null, range3, + ShardObjectHelper.newHashKeyRange("0", "199")), + ShardObjectHelper.newShard("shardId-5", "shardId-0", null, range3, + ShardObjectHelper.newHashKeyRange("200", "399")), + ShardObjectHelper.newShard("shardId-6", "shardId-2", "shardId-3", range2, + ShardObjectHelper.newHashKeyRange("500", ShardObjectHelper.MAX_HASH_KEY)), + ShardObjectHelper.newShard("shardId-7", "shardId-4", null, range4, + ShardObjectHelper.newHashKeyRange("0", "99")), + ShardObjectHelper.newShard("shardId-8", "shardId-4", null, range4, + ShardObjectHelper.newHashKeyRange("100", "199")), + ShardObjectHelper.newShard("shardId-9", "shardId-5", null, range4, + ShardObjectHelper.newHashKeyRange("200", "299")), + ShardObjectHelper.newShard("shardId-10", "shardId-5", null, range4, + ShardObjectHelper.newHashKeyRange("300", "399"))); + } + /** * Test CheckIfDescendantAndAddNewLeasesForAncestors when shardId is null */ @Test public void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() { - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); assertThat(HierarchicalShardSyncer .checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST, null, null, @@ -1589,7 +2385,7 @@ public class HierarchicalShardSyncerTest { @Test public void testCheckIfDescendantAndAddNewLeasesForAncestorsTrimmedShard() { final String shardId = "shardId-trimmed"; - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); assertThat(HierarchicalShardSyncer .checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, null, @@ -1604,7 +2400,7 @@ public class HierarchicalShardSyncerTest { final String shardId = "shardId-current"; final Set shardIdsOfCurrentLeases = new HashSet<>(Collections.singletonList(shardId)); final Map newLeaseMap = Collections.emptyMap(); - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); final Map kinesisShards = new HashMap<>(); kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, null, null, null)); @@ -1624,7 +2420,7 @@ public class HierarchicalShardSyncerTest { final String shardId = "shardId-9-1"; final Set shardIdsOfCurrentLeases = Collections.emptySet(); final Map newLeaseMap = Collections.emptyMap(); - final Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); final Map kinesisShards = new HashMap<>(); kinesisShards.put(parentShardId, ShardObjectHelper.newShard(parentShardId, null, null, null)); @@ -2313,5 +3109,4 @@ public class HierarchicalShardSyncerTest { return lease; } - } From f69398a2b207d9846aef8a707832c61fe7bfb9f1 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 3 Jun 2020 00:48:11 -0700 Subject: [PATCH 098/159] Adding dedicated consumerArn support for streams in multistreaming mode --- .../amazon/kinesis/common/StreamConfig.java | 9 ++--- .../kinesis/common/StreamIdentifier.java | 1 + .../amazon/kinesis/coordinator/Scheduler.java | 2 +- .../kinesis/retrieval/RetrievalConfig.java | 33 +++++++++++++++---- .../kinesis/retrieval/RetrievalFactory.java | 6 ++++ .../retrieval/fanout/FanOutConfig.java | 10 ++---- .../fanout/FanOutRetrievalFactory.java | 19 ++++++++--- .../kinesis/coordinator/SchedulerTest.java | 2 +- .../retrieval/fanout/FanOutConfigTest.java | 30 ++++++++++++++--- 9 files changed, 83 insertions(+), 29 deletions(-) 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 8856a4a0..b1057f13 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 @@ -15,14 +15,15 @@ package software.amazon.kinesis.common; -import lombok.Value; +import lombok.Data; import lombok.experimental.Accessors; -@Value +@Data @Accessors(fluent = true) public class StreamConfig { - StreamIdentifier streamIdentifier; - InitialPositionInStreamExtended initialPositionInStreamExtended; + private final StreamIdentifier streamIdentifier; + private final InitialPositionInStreamExtended initialPositionInStreamExtended; + private String consumerArn; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/StreamIdentifier.java index 7a416c7a..1259a609 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 @@ -63,6 +63,7 @@ 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 * @param streamIdentifierSer * @return StreamIdentifier */ 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 e2f2f852..e196920d 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 @@ -891,7 +891,6 @@ public class Scheduler implements Runnable { protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { - RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, metricsFactory); ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, @@ -902,6 +901,7 @@ public class Scheduler implements Runnable { // to gracefully complete the reading. final StreamConfig streamConfig = currentStreamConfigMap.getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); Validate.notNull(streamConfig, "StreamConfig should not be null"); + RecordsPublisher cache = retrievalConfig.retrievalFactory().createGetRecordsCache(shardInfo, streamConfig, metricsFactory); ShardConsumerArgument argument = new ShardConsumerArgument(shardInfo, streamConfig.streamIdentifier(), leaseCoordinator, 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 5f22411a..63ae7b5f 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 @@ -121,6 +121,13 @@ public class RetrievalConfig { return this; } + public RetrievalConfig retrievalSpecificConfig(RetrievalSpecificConfig retrievalSpecificConfig) { + this.retrievalSpecificConfig = retrievalSpecificConfig; + validateFanoutConfig(); + validatePollingConfig(); + return this; + } + public RetrievalFactory retrievalFactory() { if (retrievalFactory == null) { if (retrievalSpecificConfig == null) { @@ -129,22 +136,36 @@ public class RetrievalConfig { retrievalSpecificConfig = appStreamTracker.map(multiStreamTracker -> retrievalSpecificConfig, streamConfig -> ((FanOutConfig) retrievalSpecificConfig).streamName(streamConfig.streamIdentifier().streamName())); } - retrievalFactory = retrievalSpecificConfig.retrievalFactory(); } - validateConfig(); return retrievalFactory; } - private void validateConfig() { + private void validateFanoutConfig() { + // If we are in multistream mode and if retrievalSpecificConfig is an instance of FanOutConfig and if consumerArn is set throw exception. + boolean isFanoutConfig = retrievalSpecificConfig instanceof FanOutConfig; + boolean isInvalidFanoutConfig = isFanoutConfig && appStreamTracker.map( + multiStreamTracker -> ((FanOutConfig) retrievalSpecificConfig).consumerArn() != null + || ((FanOutConfig) retrievalSpecificConfig).streamName() != null, + streamConfig -> streamConfig.streamIdentifier() == null + || streamConfig.streamIdentifier().streamName() == null); + if(isInvalidFanoutConfig) { + throw new IllegalArgumentException( + "Invalid config: Either in multi-stream mode with streamName/consumerArn configured or in single-stream mode with no streamName configured"); + } + } + + private void validatePollingConfig() { boolean isPollingConfig = retrievalSpecificConfig instanceof PollingConfig; - boolean isInvalidPollingConfig = isPollingConfig && appStreamTracker.map(multiStreamTracker -> + 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: multistream enabled with streamName or single stream with no streamName"); + if (isInvalidPollingConfig) { + throw new IllegalArgumentException( + "Invalid config: Either in multi-stream mode with streamName configured or in single-stream mode with no streamName configured"); } } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java index 4c8f6b68..5703e1af 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/RetrievalFactory.java @@ -15,6 +15,7 @@ package software.amazon.kinesis.retrieval; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; @@ -24,5 +25,10 @@ import software.amazon.kinesis.metrics.MetricsFactory; public interface RetrievalFactory { GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(ShardInfo shardInfo, MetricsFactory metricsFactory); + @Deprecated RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory); + + default RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, StreamConfig streamConfig, MetricsFactory metricsFactory) { + return createGetRecordsCache(shardInfo, metricsFactory); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java index fafe7e18..9318b996 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConfig.java @@ -80,17 +80,11 @@ public class FanOutConfig implements RetrievalSpecificConfig { */ private long retryBackoffMillis = 1000; - @Override - public RetrievalFactory retrievalFactory() { - return new FanOutRetrievalFactory(kinesisClient, streamName, this::getOrCreateConsumerArn); + @Override public RetrievalFactory retrievalFactory() { + return new FanOutRetrievalFactory(kinesisClient, streamName, consumerArn, this::getOrCreateConsumerArn); } - // TODO : LTR. Need Stream Specific ConsumerArn to be passed from Customer private String getOrCreateConsumerArn(String streamName) { - if (consumerArn != null) { - return consumerArn; - } - FanOutConsumerRegistration registration = createConsumerRegistration(streamName); try { return registration.getOrCreateStreamConsumerArn(); 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 719d2e54..5796862b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java @@ -19,6 +19,7 @@ import lombok.NonNull; import lombok.RequiredArgsConstructor; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.metrics.MetricsFactory; @@ -37,8 +38,8 @@ public class FanOutRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; private final String defaultStreamName; - private final Function consumerArnProvider; - private Map streamToConsumerArnMap = new HashMap<>(); + private final String defaultConsumerName; + private final Function consumerArnCreator; @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo, @@ -48,19 +49,27 @@ public class FanOutRetrievalFactory implements RetrievalFactory { @Override public RecordsPublisher createGetRecordsCache(@NonNull final ShardInfo shardInfo, + final StreamConfig streamConfig, final MetricsFactory metricsFactory) { final Optional streamIdentifierStr = shardInfo.streamIdentifierSerOpt(); final String streamName; if(streamIdentifierStr.isPresent()) { streamName = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()).streamName(); return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), - streamToConsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply), + getOrCreateConsumerArn(streamName, streamConfig.consumerArn()), streamIdentifierStr.get()); } else { - streamName = defaultStreamName; return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), - streamToConsumerArnMap.computeIfAbsent(streamName, consumerArnProvider::apply)); + getOrCreateConsumerArn(defaultStreamName, defaultConsumerName)); } + } + @Override + public RecordsPublisher createGetRecordsCache(ShardInfo shardInfo, MetricsFactory metricsFactory) { + throw new UnsupportedOperationException("FanoutRetrievalFactory needs StreamConfig Info"); + } + + private String getOrCreateConsumerArn(String streamName, String consumerArn) { + return consumerArn != null ? consumerArn : consumerArnCreator.apply(streamName); } } 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 e5a76ce3..1d24ae68 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 @@ -191,7 +191,7 @@ public class SchedulerTest { when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); - when(retrievalFactory.createGetRecordsCache(any(ShardInfo.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, 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 21228c75..d8953245 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 @@ -25,12 +25,14 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.metrics.MetricsFactory; @@ -50,6 +52,13 @@ public class FanOutConfigTest { private FanOutConsumerRegistration consumerRegistration; @Mock private KinesisAsyncClient kinesisClient; + @Mock + private StreamConfig streamConfig; + + @Before + public void setup() { + when(streamConfig.consumerArn()).thenReturn(null); + } @Test public void testNoRegisterIfConsumerArnSet() throws Exception { @@ -68,11 +77,24 @@ public class FanOutConfigTest { 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, mock(MetricsFactory.class)); + retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(retrievalFactory, not(nullValue())); verify(consumerRegistration).getOrCreateStreamConsumerArn(); } + @Test + public void testRegisterNotCalledWhenConsumerArnSetInMultiStreamMode() throws Exception { + when(streamConfig.consumerArn()).thenReturn("consumerArn"); + FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) + .streamName(TEST_STREAM_NAME); + RetrievalFactory retrievalFactory = config.retrievalFactory(); + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of("account:stream:12345")).when(shardInfo).streamIdentifierSerOpt(); + retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); + assertThat(retrievalFactory, not(nullValue())); + verify(consumerRegistration, never()).getOrCreateStreamConsumerArn(); + } + @Test public void testDependencyExceptionInConsumerCreation() throws Exception { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) @@ -94,7 +116,7 @@ public class FanOutConfigTest { RetrievalFactory factory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; @@ -109,7 +131,7 @@ public class FanOutConfigTest { RetrievalFactory factory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; assertThat(testingConfig.stream, equalTo(TEST_STREAM_NAME)); @@ -123,7 +145,7 @@ public class FanOutConfigTest { RetrievalFactory factory = config.retrievalFactory(); ShardInfo shardInfo = mock(ShardInfo.class); doReturn(Optional.empty()).when(shardInfo).streamIdentifierSerOpt(); - factory.createGetRecordsCache(shardInfo, mock(MetricsFactory.class)); + factory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); assertThat(factory, not(nullValue())); TestingConfig testingConfig = (TestingConfig) config; From b5d0301b315a7a0c6bb12c2ad38e1cedbcde4876 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 3 Jun 2020 01:08:13 -0700 Subject: [PATCH 099/159] Adding stream information to Fanout Consumer Registration logs --- .../fanout/FanOutConsumerRegistration.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java index 0519390c..9bcdd83c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutConsumerRegistration.java @@ -76,7 +76,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { try { response = describeStreamConsumer(); } catch (ResourceNotFoundException e) { - log.info("StreamConsumer not found, need to create it."); + log.info("{} : StreamConsumer not found, need to create it.", streamName); } // 2. If not, register consumer @@ -92,7 +92,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { break; } catch (LimitExceededException e) { // TODO: Figure out internal service exceptions - log.debug("RegisterStreamConsumer call got throttled will retry."); + log.debug("{} : RegisterStreamConsumer call got throttled will retry.", streamName); finalException = e; } retries--; @@ -104,7 +104,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { } } catch (ResourceInUseException e) { // Consumer is present, call DescribeStreamConsumer - log.debug("Got ResourceInUseException consumer exists, will call DescribeStreamConsumer again."); + log.debug("{} : Got ResourceInUseException consumer exists, will call DescribeStreamConsumer again.", streamName); response = describeStreamConsumer(); } } @@ -160,17 +160,17 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { while (!ConsumerStatus.ACTIVE.equals(status) && retries > 0) { status = describeStreamConsumer().consumerDescription().consumerStatus(); retries--; - log.info(String.format("Waiting for StreamConsumer %s to have ACTIVE status...", streamConsumerName)); + log.info("{} : Waiting for StreamConsumer {} to have ACTIVE status...", streamName, streamConsumerName); Thread.sleep(retryBackoffMillis); } } catch (InterruptedException ie) { - log.debug("Thread was interrupted while fetching StreamConsumer status, moving on."); + log.debug("{} : Thread was interrupted while fetching StreamConsumer status, moving on.", streamName); } if (!ConsumerStatus.ACTIVE.equals(status)) { final String message = String.format( - "Status of StreamConsumer %s, was not ACTIVE after all retries. Was instead %s.", - streamConsumerName, status); + "%s : Status of StreamConsumer %s, was not ACTIVE after all retries. Was instead %s.", + streamName, streamConsumerName, status); log.error(message); throw new IllegalStateException(message); } @@ -211,7 +211,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { throw new DependencyException(e); } } catch (LimitExceededException e) { - log.info("Throttled while calling {} API, will backoff.", apiName); + log.info("{} : Throttled while calling {} API, will backoff.", streamName, apiName); try { Thread.sleep(retryBackoffMillis + (long) (Math.random() * 100)); } catch (InterruptedException ie) { @@ -224,7 +224,7 @@ public class FanOutConsumerRegistration implements ConsumerRegistration { if (finalException == null) { throw new IllegalStateException( - String.format("Finished all retries and no exception was caught while calling %s", apiName)); + String.format("%s : Finished all retries and no exception was caught while calling %s", streamName, apiName)); } throw finalException; From 700c7a563ae17c9036a2060b74a8c5436b7d8314 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Sun, 7 Jun 2020 21:17:24 -0700 Subject: [PATCH 100/159] Adding unit test case --- .../kinesis/retrieval/fanout/FanOutConfigTest.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) 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 d8953245..4fee3d08 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/fanout/FanOutConfigTest.java @@ -95,6 +95,18 @@ public class FanOutConfigTest { verify(consumerRegistration, never()).getOrCreateStreamConsumerArn(); } + @Test + public void testRegisterCalledWhenConsumerArnNotSetInMultiStreamMode() throws Exception { + FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) + .streamName(TEST_STREAM_NAME); + RetrievalFactory retrievalFactory = config.retrievalFactory(); + ShardInfo shardInfo = mock(ShardInfo.class); + doReturn(Optional.of("account:stream:12345")).when(shardInfo).streamIdentifierSerOpt(); + retrievalFactory.createGetRecordsCache(shardInfo, streamConfig, mock(MetricsFactory.class)); + assertThat(retrievalFactory, not(nullValue())); + verify(consumerRegistration).getOrCreateStreamConsumerArn(); + } + @Test public void testDependencyExceptionInConsumerCreation() throws Exception { FanOutConfig config = new TestingConfig(kinesisClient).applicationName(TEST_APPLICATION_NAME) From b323e7c48704c07cb14763076c90302610e83d5a Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Sun, 7 Jun 2020 20:35:25 -0700 Subject: [PATCH 101/159] Introducing dedicated shard syncer for each of the streamconfig --- .../amazon/kinesis/coordinator/Scheduler.java | 7 ++- .../leases/HierarchicalShardSyncer.java | 8 ++-- .../kinesis/leases/LeaseManagementConfig.java | 18 ++------ .../DynamoDBLeaseManagementFactory.java | 45 ++++++++++--------- .../kinesis/coordinator/SchedulerTest.java | 16 ++++++- .../leases/HierarchicalShardSyncerTest.java | 2 +- 6 files changed, 49 insertions(+), 47 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 e196920d..38a8131a 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 @@ -160,7 +160,7 @@ public class Scheduler implements Runnable { private final Function shardDetectorProvider; private final boolean ignoreUnexpetedChildShards; private final AggregatorUtil aggregatorUtil; - private final HierarchicalShardSyncer hierarchicalShardSyncer; + private final Function hierarchicalShardSyncerProvider; private final long schedulerInitializationBackoffTimeMillis; private final LeaderDecider leaderDecider; private final Map staleStreamDeletionMap = new HashMap<>(); @@ -284,8 +284,7 @@ public class Scheduler implements Runnable { this.shardDetectorProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).shardDetector(); this.ignoreUnexpetedChildShards = this.leaseManagementConfig.ignoreUnexpectedChildShards(); this.aggregatorUtil = this.lifecycleConfig.aggregatorUtil(); - // TODO : LTR : Check if this needs to be per stream. - this.hierarchicalShardSyncer = leaseManagementConfig.hierarchicalShardSyncer(isMultiStreamMode); + this.hierarchicalShardSyncerProvider = streamConfig -> createOrGetShardSyncTaskManager(streamConfig).hierarchicalShardSyncer(); this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager( leaseManagementConfig.workerIdentifier(), leaderDecider, leaseRefresher, currentStreamConfigMap, @@ -922,7 +921,7 @@ public class Scheduler implements Runnable { ignoreUnexpetedChildShards, shardDetectorProvider.apply(streamConfig), aggregatorUtil, - hierarchicalShardSyncer, + hierarchicalShardSyncerProvider.apply(streamConfig), metricsFactory); 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/HierarchicalShardSyncer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java index a2700097..68a1701d 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 @@ -74,7 +74,7 @@ public class HierarchicalShardSyncer { private final boolean isMultiStreamMode; - private String streamIdentifier = ""; + private final String streamIdentifier; private static final String MIN_HASH_KEY = BigInteger.ZERO.toString(); private static final String MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE).toString(); @@ -84,10 +84,12 @@ public class HierarchicalShardSyncer { public HierarchicalShardSyncer() { isMultiStreamMode = false; + streamIdentifier = "SingleStreamMode"; } - public HierarchicalShardSyncer(final boolean isMultiStreamMode) { + public HierarchicalShardSyncer(final boolean isMultiStreamMode, final String streamIdentifier) { this.isMultiStreamMode = isMultiStreamMode; + this.streamIdentifier = streamIdentifier; } private static final BiFunction shardIdFromLeaseDeducer = @@ -118,7 +120,6 @@ public class HierarchicalShardSyncer { final MetricsScope scope, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { - this.streamIdentifier = shardDetector.streamIdentifier().serialize(); final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, garbageCollectLeases, @@ -132,7 +133,6 @@ public class HierarchicalShardSyncer { final MetricsScope scope, final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - this.streamIdentifier = shardDetector.streamIdentifier().serialize(); //TODO: Need to add multistream support for this https://sim.amazon.com/issues/KinesisLTR-191 if (!CollectionUtils.isNullOrEmpty(latestShards)) { 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 acaa8de0..789a3008 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 @@ -276,19 +276,6 @@ public class LeaseManagementConfig { return hierarchicalShardSyncer; } - /** - * Vends HierarchicalShardSyncer based on MultiStreamingMode. With MultiStreamMode shard syncer creates - * leases to accommodate more than one stream. - * @param isMultiStreamingMode - * @return HierarchicalShardSyncer - */ - public HierarchicalShardSyncer hierarchicalShardSyncer(boolean isMultiStreamingMode) { - if(hierarchicalShardSyncer == null) { - hierarchicalShardSyncer = new HierarchicalShardSyncer(isMultiStreamingMode); - } - return hierarchicalShardSyncer; - } - @Deprecated public LeaseManagementFactory leaseManagementFactory() { if (leaseManagementFactory == null) { @@ -351,12 +338,13 @@ public class LeaseManagementConfig { cacheMissWarningModulus(), initialLeaseTableReadCapacity(), initialLeaseTableWriteCapacity(), - hierarchicalShardSyncer(isMultiStreamingMode), + hierarchicalShardSyncer(), tableCreatorCallback(), dynamoDbRequestTimeout(), billingMode(), leaseSerializer, - customShardDetectorProvider()); + customShardDetectorProvider(), + isMultiStreamingMode); } return leaseManagementFactory; } 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 44879c1c..c1b250a4 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 @@ -55,7 +55,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { @NonNull private final ExecutorService executorService; @NonNull - private final HierarchicalShardSyncer hierarchicalShardSyncer; + private final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer; @NonNull private final LeaseSerializer leaseSerializer; @NonNull @@ -82,6 +82,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { private final TableCreatorCallback tableCreatorCallback; private final Duration dynamoDbRequestTimeout; private final BillingMode billingMode; + private final boolean isMultiStreamMode; /** * Constructor. @@ -207,7 +208,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param hierarchicalShardSyncer + * @param deprecatedHierarchicalShardSyncer * @param tableCreatorCallback */ @Deprecated @@ -221,14 +222,14 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback) { + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - hierarchicalShardSyncer, tableCreatorCallback, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); + deprecatedHierarchicalShardSyncer, tableCreatorCallback, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); } /** @@ -257,7 +258,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param hierarchicalShardSyncer + * @param deprecatedHierarchicalShardSyncer * @param tableCreatorCallback * @param dynamoDbRequestTimeout */ @@ -272,7 +273,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, @@ -280,7 +281,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, BillingMode.PROVISIONED); + deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, BillingMode.PROVISIONED); } /** @@ -309,7 +310,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param hierarchicalShardSyncer + * @param deprecatedHierarchicalShardSyncer * @param tableCreatorCallback * @param dynamoDbRequestTimeout * @param billingMode @@ -325,7 +326,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode) { this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName, @@ -334,7 +335,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); + deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); } /** @@ -362,7 +363,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param hierarchicalShardSyncer + * @param deprecatedHierarchicalShardSyncer * @param tableCreatorCallback * @param dynamoDbRequestTimeout * @param billingMode @@ -376,7 +377,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer) { this(kinesisClient, dynamoDBClient, tableName, workerIdentifier, executorService, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, @@ -384,8 +385,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer, - null); + deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer, + null, false); this.streamConfig = streamConfig; } @@ -412,11 +413,13 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param hierarchicalShardSyncer + * @param deprecatedHierarchicalShardSyncer * @param tableCreatorCallback * @param dynamoDbRequestTimeout * @param billingMode * @param leaseSerializer + * @param customShardDetectorProvider + * @param isMultiStreamMode */ public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, @@ -427,9 +430,9 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer, - Function customShardDetectorProvider) { + Function customShardDetectorProvider, boolean isMultiStreamMode) { this.kinesisClient = kinesisClient; this.dynamoDBClient = dynamoDBClient; this.tableName = tableName; @@ -451,12 +454,13 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.cacheMissWarningModulus = cacheMissWarningModulus; this.initialLeaseTableReadCapacity = initialLeaseTableReadCapacity; this.initialLeaseTableWriteCapacity = initialLeaseTableWriteCapacity; - this.hierarchicalShardSyncer = hierarchicalShardSyncer; + this.deprecatedHierarchicalShardSyncer = deprecatedHierarchicalShardSyncer; this.tableCreatorCallback = tableCreatorCallback; this.dynamoDbRequestTimeout = dynamoDbRequestTimeout; this.billingMode = billingMode; this.leaseSerializer = leaseSerializer; this.customShardDetectorProvider = customShardDetectorProvider; + this.isMultiStreamMode = isMultiStreamMode; } @Override @@ -481,8 +485,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, - executorService, - hierarchicalShardSyncer, + executorService, deprecatedHierarchicalShardSyncer, metricsFactory); } @@ -501,7 +504,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, executorService, - hierarchicalShardSyncer, + new HierarchicalShardSyncer(isMultiStreamMode, streamConfig.streamIdentifier().toString()), metricsFactory); } 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 1d24ae68..a1601eda 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 @@ -73,6 +73,7 @@ import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.KinesisClientLibException; import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException; +import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseManagementFactory; @@ -153,11 +154,13 @@ public class SchedulerTest { @Mock private MultiStreamTracker multiStreamTracker; - private Map shardSyncTaskManagerMap = new HashMap<>(); - private Map shardDetectorMap = new HashMap<>(); + private Map shardSyncTaskManagerMap; + private Map shardDetectorMap; @Before public void setup() { + shardSyncTaskManagerMap = new HashMap<>(); + shardDetectorMap = new HashMap<>(); shardRecordProcessorFactory = new TestShardRecordProcessorFactory(); checkpointConfig = new CheckpointConfig().checkpointFactory(new TestKinesisCheckpointFactory()); @@ -190,6 +193,7 @@ public class SchedulerTest { }); when(leaseCoordinator.leaseRefresher()).thenReturn(dynamoDBLeaseRefresher); 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(shardDetector.streamIdentifier()).thenReturn(mock(StreamIdentifier.class)); @@ -334,6 +338,8 @@ public class SchedulerTest { scheduler.initialize(); shardDetectorMap.values().stream() .forEach(shardDetector -> verify(shardDetector, times(1)).listShards()); + shardSyncTaskManagerMap.values().stream() + .forEach(shardSyncTM -> verify(shardSyncTM, times(1)).hierarchicalShardSyncer()); } @Test @@ -352,6 +358,10 @@ public class SchedulerTest { .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()); } @@ -1035,6 +1045,8 @@ public class SchedulerTest { shardSyncTaskManagerMap.put(streamConfig.streamIdentifier(), shardSyncTaskManager); shardDetectorMap.put(streamConfig.streamIdentifier(), shardDetector); when(shardSyncTaskManager.shardDetector()).thenReturn(shardDetector); + final HierarchicalShardSyncer hierarchicalShardSyncer = new HierarchicalShardSyncer(); + when(shardSyncTaskManager.hierarchicalShardSyncer()).thenReturn(hierarchicalShardSyncer); when(shardDetector.streamIdentifier()).thenReturn(streamConfig.streamIdentifier()); when(shardSyncTaskManager.callShardSyncTask()).thenReturn(new TaskResult(null)); if(shardSyncFirstAttemptFailure) { 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 096bf33a..1be28b1d 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 @@ -112,7 +112,7 @@ public class HierarchicalShardSyncerTest { } private void setupMultiStream() { - hierarchicalShardSyncer = new HierarchicalShardSyncer(true); + hierarchicalShardSyncer = new HierarchicalShardSyncer(true, STREAM_IDENTIFIER); when(shardDetector.streamIdentifier()).thenReturn(StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); } From c7cdbd5d8bd8d5d251858963346eb57f5ee97f0c Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Thu, 4 Jun 2020 03:49:07 -0400 Subject: [PATCH 102/159] pom change --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f21b230e..4e8da722 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.11.8-SNAPSHOT + 2.10.65-SNAPSHOT From 2f1838483c361a0e61b6e79fc86a8ac3a3d92158 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 6 May 2020 08:51:06 -0400 Subject: [PATCH 103/159] Adding Lease cleanup in shutdown task. --- .../kinesis/common/LeaseCleanupConfig.java | 41 +++ .../amazon/kinesis/coordinator/Scheduler.java | 42 +-- .../kinesis/leases/LeaseCleanupManager.java | 338 +++++++++++++++++ .../kinesis/leases/LeaseManagementConfig.java | 23 +- .../leases/LeaseManagementFactory.java | 2 + .../DynamoDBLeaseManagementFactory.java | 44 ++- .../exceptions/LeasePendingDeletion.java | 35 ++ .../kinesis/lifecycle/ConsumerStates.java | 6 +- .../lifecycle/ShardConsumerArgument.java | 2 + .../kinesis/lifecycle/ShutdownTask.java | 46 ++- .../kinesis/coordinator/SchedulerTest.java | 32 +- .../leases/LeaseCleanupManagerTest.java | 347 ++++++++++++++++++ .../amazon/kinesis/leases/LeaseHelper.java | 44 +++ .../kinesis/lifecycle/ConsumerStatesTest.java | 7 +- .../kinesis/lifecycle/ShutdownTaskTest.java | 54 ++- 15 files changed, 969 insertions(+), 94 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java create mode 100644 amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java new file mode 100644 index 00000000..b2582d45 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/common/LeaseCleanupConfig.java @@ -0,0 +1,41 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.common; + +import lombok.Builder; +import lombok.Getter; +import lombok.experimental.Accessors; + +/** + * Configuration for lease cleanup. + */ +@Builder +@Getter +@Accessors(fluent=true) +public class LeaseCleanupConfig { + /** + * Interval at which to run lease cleanup thread. + */ + private final long leaseCleanupIntervalMillis; + /** + * Interval at which to check if a lease is completed or not. + */ + private final long completedLeaseCleanupIntervalMillis; + /** + * Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. + */ + private final long garbageLeaseCleanupIntervalMillis; +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java index 38a8131a..3e74e23b 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 @@ -58,6 +58,7 @@ import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseRefresher; @@ -164,6 +165,7 @@ public class Scheduler implements Runnable { private final long schedulerInitializationBackoffTimeMillis; private final LeaderDecider leaderDecider; private final Map staleStreamDeletionMap = new HashMap<>(); + private final LeaseCleanupManager leaseCleanupManager; // Holds consumers for shards the worker is currently tracking. Key is shard // info, value is ShardConsumer. @@ -289,6 +291,8 @@ public class Scheduler implements Runnable { this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager( leaseManagementConfig.workerIdentifier(), leaderDecider, leaseRefresher, currentStreamConfigMap, shardSyncTaskManagerProvider, isMultiStreamMode); + this.leaseCleanupManager = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer, isMultiStreamMode) + .createLeaseCleanupManager(metricsFactory); } /** @@ -341,6 +345,13 @@ public class Scheduler implements Runnable { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); } + if (!leaseCleanupManager.isRunning()) { + log.info("Starting LeaseCleanupManager."); + leaseCleanupManager.start(); + } else { + log.info("LeaseCleanupManager is already running. No need to start it"); + } + // If we reach this point, then we either skipped the lease sync or did not have any exception // for any of the shard sync in the previous attempt. if (!leaseCoordinator.isRunning()) { @@ -397,29 +408,14 @@ public class Scheduler implements Runnable { void runProcessLoop() { try { Set assignedShards = new HashSet<>(); - final Set completedShards = new HashSet<>(); for (ShardInfo shardInfo : getShardInfoForAssignments()) { ShardConsumer shardConsumer = createOrGetShardConsumer(shardInfo, - processorConfig.shardRecordProcessorFactory()); + processorConfig.shardRecordProcessorFactory(), leaseCleanupManager); - if (shardConsumer.isShutdown() && shardConsumer.shutdownReason().equals(ShutdownReason.SHARD_END)) { - completedShards.add(shardInfo); - } else { - shardConsumer.executeLifecycle(); - } + shardConsumer.executeLifecycle(); assignedShards.add(shardInfo); } - for (ShardInfo completedShard : completedShards) { - final StreamIdentifier streamIdentifier = getStreamIdentifier(completedShard.streamIdentifierSerOpt()); - final StreamConfig streamConfig = currentStreamConfigMap - .getOrDefault(streamIdentifier, getDefaultStreamConfig(streamIdentifier)); - if (createOrGetShardSyncTaskManager(streamConfig).submitShardSyncTask()) { - log.info("{} : Found completed shard, initiated new ShardSyncTak for {} ", - streamIdentifier.serialize(), completedShard.toString()); - } - } - // clean up shard consumers for unassigned shards cleanupShardConsumers(assignedShards); @@ -868,7 +864,8 @@ public class Scheduler implements Runnable { * @return ShardConsumer for the shard */ ShardConsumer createOrGetShardConsumer(@NonNull final ShardInfo shardInfo, - @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { + @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory, + @NonNull final LeaseCleanupManager leaseCleanupManager) { ShardConsumer consumer = shardInfoShardConsumerMap.get(shardInfo); // Instantiate a new consumer if we don't have one, or the one we // had was from an earlier @@ -877,7 +874,7 @@ public class Scheduler implements Runnable { // completely processed (shutdown reason terminate). if ((consumer == null) || (consumer.isShutdown() && consumer.shutdownReason().equals(ShutdownReason.LEASE_LOST))) { - consumer = buildConsumer(shardInfo, shardRecordProcessorFactory); + consumer = buildConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager); shardInfoShardConsumerMap.put(shardInfo, consumer); slog.infoForce("Created new shardConsumer for : " + shardInfo); } @@ -889,12 +886,14 @@ public class Scheduler implements Runnable { } protected ShardConsumer buildConsumer(@NonNull final ShardInfo shardInfo, - @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory) { + @NonNull final ShardRecordProcessorFactory shardRecordProcessorFactory, + @NonNull final LeaseCleanupManager leaseCleanupManager) { ShardRecordProcessorCheckpointer checkpointer = coordinatorConfig.coordinatorFactory().createRecordProcessorCheckpointer(shardInfo, checkpoint); // The only case where streamName is not available will be when multistreamtracker not set. In this case, // get the default stream name for the single stream application. final StreamIdentifier streamIdentifier = getStreamIdentifier(shardInfo.streamIdentifierSerOpt()); + // Irrespective of single stream app or multi stream app, streamConfig should always be available. // If we have a shardInfo, that is not present in currentStreamConfigMap for whatever reason, then return default stream config // to gracefully complete the reading. @@ -922,7 +921,8 @@ public class Scheduler implements Runnable { shardDetectorProvider.apply(streamConfig), aggregatorUtil, hierarchicalShardSyncerProvider.apply(streamConfig), - metricsFactory); + metricsFactory, + leaseCleanupManager); 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/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java new file mode 100644 index 00000000..ba108748 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java @@ -0,0 +1,338 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.Value; +import lombok.experimental.Accessors; +import lombok.extern.slf4j.Slf4j; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +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.ResourceNotFoundException; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.kinesis.common.FutureUtils; +import software.amazon.kinesis.common.KinesisRequestsBuilder; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; +import software.amazon.kinesis.leases.exceptions.InvalidStateException; +import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; +import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.retrieval.AWSExceptionManager; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.time.Duration; +import java.util.HashSet; +import java.util.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * Helper class to cleanup of any expired/closed shard leases. It will cleanup leases periodically as defined by + * {@link LeaseManagementConfig#leaseCleanupConfig()} asynchronously. + */ +@Accessors(fluent=true) +@Slf4j +@RequiredArgsConstructor +@EqualsAndHashCode +public class LeaseCleanupManager { + @NonNull + private final LeaseCoordinator leaseCoordinator; + @NonNull + private final KinesisAsyncClient kinesisClient; + @NonNull + private final MetricsFactory metricsFactory; + @NonNull + private final Duration maxFutureWait; + @NonNull + private final ScheduledExecutorService deletionThreadPool; + private final boolean cleanupLeasesUponShardCompletion; + private final long leaseCleanupIntervalMillis; + private final long completedLeaseCleanupIntervalMillis; + private final long garbageLeaseCleanupIntervalMillis; + private final Stopwatch completedLeaseStopwatch = Stopwatch.createUnstarted(); + private final Stopwatch garbageLeaseStopwatch = Stopwatch.createUnstarted(); + + private final Queue deletionQueue = new ConcurrentLinkedQueue<>(); + + private static final int MAX_RECORDS = 1; + private static final long INITIAL_DELAY = 0L; + + @Getter + private volatile boolean isRunning = false; + + /** + * Starts the lease cleanup thread, which is scheduled periodically as specified by + * {@link LeaseCleanupManager#leaseCleanupIntervalMillis} + */ + public void start() { + log.debug("Starting lease cleanup thread."); + completedLeaseStopwatch.start(); + garbageLeaseStopwatch.start(); + + deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, + TimeUnit.MILLISECONDS); + } + + /** + * Enqueues a lease for deletion. + * @param leasePendingDeletion + */ + public void enqueueForDeletion(LeasePendingDeletion leasePendingDeletion) { + final Lease lease = leasePendingDeletion.lease(); + if (lease == null) { + log.warn("Cannot enqueue lease {} for deferred deletion - instance doesn't hold the lease for that shard.", + lease.leaseKey()); + } else { + //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597. + if (!deletionQueue.contains(leasePendingDeletion)) { + log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey()); + deletionQueue.add(leasePendingDeletion); + } else { + log.warn("Lease {} is already pending deletion, not enqueueing for deletion.", lease.leaseKey()); + } + } + } + + /** + * Returns how many leases are currently waiting in the queue pending deletion. + * @return number of leases pending deletion. + */ + public int leasesPendingDeletion() { + return deletionQueue.size(); + } + + private boolean timeToCheckForCompletedShard() { + return completedLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= completedLeaseCleanupIntervalMillis; + } + + private boolean timeToCheckForGarbageShard() { + return garbageLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= garbageLeaseCleanupIntervalMillis; + } + + private LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion) throws TimeoutException, + InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { + final Lease lease = leasePendingDeletion.lease(); + final ShardInfo shardInfo = leasePendingDeletion.shardInfo(); + final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier(); + + final AWSExceptionManager exceptionManager = createExceptionManager(); + + boolean cleanedUpCompletedLease = false; + boolean cleanedUpGarbageLease = false; + boolean alreadyCheckedForGarbageCollection = false; + + try { + if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard()) { + Set childShardKeys = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey()).childShardIds(); + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + try { + childShardKeys = getChildShardsFromService(shardInfo, streamIdentifier); + updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } finally { + alreadyCheckedForGarbageCollection = true; + } + } + cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys); + } + + if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard()) { + try { + getChildShardsFromService(shardInfo, streamIdentifier); + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } + } + } catch (ResourceNotFoundException e) { + cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease); + } + + return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease); + } + + private Set getChildShardsFromService(ShardInfo shardInfo, StreamIdentifier streamIdentifier) + throws InterruptedException, ExecutionException, TimeoutException { + final GetShardIteratorRequest getShardIteratorRequest = KinesisRequestsBuilder.getShardIteratorRequestBuilder() + .streamName(streamIdentifier.streamName()) + .shardIteratorType(ShardIteratorType.LATEST) + .shardId(shardInfo.shardId()) + .build(); + + final GetShardIteratorResponse getShardIteratorResponse = + FutureUtils.resolveOrCancelFuture(kinesisClient.getShardIterator(getShardIteratorRequest), maxFutureWait); + + final GetRecordsRequest getRecordsRequest = KinesisRequestsBuilder.getRecordsRequestBuilder() + .shardIterator(getShardIteratorResponse.shardIterator()) + .limit(MAX_RECORDS) + .build(); + + final GetRecordsResponse getRecordsResponse = + FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(getRecordsRequest), maxFutureWait); + + return getRecordsResponse.childShards().stream().map(c -> c.shardId()).collect(Collectors.toSet()); + } + + + // A lease that ended with SHARD_END from ResourceNotFoundException is safe to delete if it no longer exists in the + // stream (known explicitly from ResourceNotFound being thrown when processing this shard), + private boolean cleanupLeaseForGarbageShard(Lease lease) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + log.info("Deleting lease {} as it is not present in the stream.", lease); + leaseCoordinator.leaseRefresher().deleteLease(lease); + return true; + } + + private boolean allParentShardLeasesDeleted(Lease lease, ShardInfo shardInfo) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + for (String parentShard : lease.parentShardIds()) { + final Lease parentLease = leaseCoordinator.leaseRefresher().getLease(ShardInfo.getLeaseKey(shardInfo, parentShard)); + + if (parentLease != null) { + log.warn("Lease {} has a parent lease {} which is still present in the lease table, skipping deletion " + + "for this lease.", lease, parentLease); + return false; + } + } + return true; + } + + // We should only be deleting the current shard's lease if + // 1. All of its children are currently being processed, i.e their checkpoint is not TRIM_HORIZON or AT_TIMESTAMP. + // 2. Its parent shard lease(s) have already been deleted. + private boolean cleanupLeaseForCompletedShard(Lease lease, ShardInfo shardInfo, Set childShardKeys) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, IllegalStateException { + final Set processedChildShardLeases = new HashSet<>(); + + for (String childShardKey : childShardKeys) { + final Lease childShardLease = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(childShardKey)).orElseThrow( + () -> new IllegalStateException("Child lease " + childShardKey + " for completed shard not found in " + + "lease table - not cleaning up lease " + lease)); + + if (!childShardLease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) + && !childShardLease.checkpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) { + processedChildShardLeases.add(childShardLease.leaseKey()); + } + } + + if (!allParentShardLeasesDeleted(lease, shardInfo) || !Objects.equals(childShardKeys, processedChildShardLeases)) { + return false; + } + + log.info("Deleting lease {} as it has been completely processed and processing of child shard(s) has begun.", + lease); + leaseCoordinator.leaseRefresher().deleteLease(lease); + + return true; + } + + private void updateLeaseWithChildShards(LeasePendingDeletion leasePendingDeletion, Set childShardKeys) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final Lease updatedLease = leasePendingDeletion.lease(); + updatedLease.childShardIds(childShardKeys); + + leaseCoordinator.leaseRefresher().updateLeaseWithMetaInfo(updatedLease, UpdateField.CHILD_SHARDS); + } + + private AWSExceptionManager createExceptionManager() { + final AWSExceptionManager exceptionManager = new AWSExceptionManager(); + exceptionManager.add(ResourceNotFoundException.class, t -> t); + + return exceptionManager; + } + + @VisibleForTesting + void cleanupLeases() { + if (deletionQueue.isEmpty()) { + log.debug("No leases pending deletion."); + } else if (timeToCheckForCompletedShard() | timeToCheckForGarbageShard()) { + final Queue failedDeletions = new ConcurrentLinkedQueue<>(); + boolean completedLeaseCleanedUp = false; + boolean garbageLeaseCleanedUp = false; + + log.debug("Attempting to clean up {} lease(s).", deletionQueue.size()); + + while (!deletionQueue.isEmpty()) { + final LeasePendingDeletion leasePendingDeletion = deletionQueue.poll(); + final String leaseKey = leasePendingDeletion.lease().leaseKey(); + final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier(); + boolean deletionFailed = true; + try { + final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion); + completedLeaseCleanedUp |= leaseCleanupResult.cleanedUpCompletedLease(); + garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease(); + + if (leaseCleanupResult.leaseCleanedUp()) { + log.debug("Successfully cleaned up lease {} for {}", leaseKey, streamIdentifier); + deletionFailed = false; + } + } catch (Exception e) { + log.error("Failed to cleanup lease {} for {}. Will re-enqueue for deletion and retry on next " + + "scheduled execution.", leaseKey, streamIdentifier, e); + } + + if (deletionFailed) { + log.debug("Did not cleanup lease {} for {}. Re-enqueueing for deletion.", leaseKey, streamIdentifier); + failedDeletions.add(leasePendingDeletion); + } + } + + if (completedLeaseCleanedUp) { + log.debug("At least one completed lease was cleaned up - restarting interval"); + completedLeaseStopwatch.reset().start(); + } + + if (garbageLeaseCleanedUp) { + log.debug("At least one garbage lease was cleaned up - restarting interval"); + garbageLeaseStopwatch.reset().start(); + } + + deletionQueue.addAll(failedDeletions); + } + } + + private class LeaseCleanupThread implements Runnable { + @Override + public void run() { + cleanupLeases(); + } + } + + @Value + private class LeaseCleanupResult { + boolean cleanedUpCompletedLease; + boolean cleanedUpGarbageLease; + + public boolean leaseCleanedUp() { + return cleanedUpCompletedLease | cleanedUpGarbageLease; + } + } +} 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 789a3008..82c02060 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 @@ -33,6 +33,7 @@ import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.LeaseCleanupConfig; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.leases.dynamodb.DynamoDBLeaseManagementFactory; import software.amazon.kinesis.leases.dynamodb.TableCreatorCallback; @@ -48,6 +49,16 @@ public class LeaseManagementConfig { public static final Duration DEFAULT_REQUEST_TIMEOUT = Duration.ofMinutes(1); + public static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(1).toMillis(); + public static final long DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(5).toMillis(); + public static final long DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(30).toMillis(); + + public static final LeaseCleanupConfig DEFAULT_LEASE_CLEANUP_CONFIG = LeaseCleanupConfig.builder() + .leaseCleanupIntervalMillis(DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS) + .completedLeaseCleanupIntervalMillis(DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS) + .garbageLeaseCleanupIntervalMillis(DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS) + .build(); + /** * Name of the table to use in DynamoDB * @@ -108,6 +119,15 @@ public class LeaseManagementConfig { */ private boolean cleanupLeasesUponShardCompletion = true; + /** + * Configuration for lease cleanup in {@link LeaseCleanupManager}. + * + *

Default lease cleanup interval value: 1 minute.

+ *

Default completed lease cleanup threshold: 5 minute.

+ *

Default garbage lease cleanup threshold: 30 minute.

+ */ + private final LeaseCleanupConfig leaseCleanupConfig = DEFAULT_LEASE_CLEANUP_CONFIG; + /** * The max number of leases (shards) this worker should process. * This can be useful to avoid overloading (and thrashing) a worker when a host has resource constraints @@ -344,7 +364,8 @@ public class LeaseManagementConfig { billingMode(), leaseSerializer, customShardDetectorProvider(), - isMultiStreamingMode); + isMultiStreamingMode, + leaseCleanupConfig()); } return leaseManagementFactory; } 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 37f66258..ecf9b390 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementFactory.java @@ -39,4 +39,6 @@ public interface LeaseManagementFactory { throw new UnsupportedOperationException(); } + LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory); + } 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 c1b250a4..7d374de5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -17,6 +17,7 @@ package software.amazon.kinesis.leases.dynamodb; import java.time.Duration; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.function.Function; import lombok.Data; import lombok.NonNull; @@ -25,10 +26,12 @@ import software.amazon.awssdk.services.dynamodb.model.BillingMode; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.LeaseCleanupConfig; import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.KinesisShardDetector; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; import software.amazon.kinesis.leases.LeaseManagementFactory; @@ -83,6 +86,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { private final Duration dynamoDbRequestTimeout; private final BillingMode billingMode; private final boolean isMultiStreamMode; + private final LeaseCleanupConfig leaseCleanupConfig; /** * Constructor. @@ -208,7 +212,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param deprecatedHierarchicalShardSyncer + * @param hierarchicalShardSyncer * @param tableCreatorCallback */ @Deprecated @@ -222,14 +226,14 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback) { + final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewalThreads, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - deprecatedHierarchicalShardSyncer, tableCreatorCallback, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); + hierarchicalShardSyncer, tableCreatorCallback, LeaseManagementConfig.DEFAULT_REQUEST_TIMEOUT); } /** @@ -258,7 +262,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param deprecatedHierarchicalShardSyncer + * @param hierarchicalShardSyncer * @param tableCreatorCallback * @param dynamoDbRequestTimeout */ @@ -273,7 +277,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout) { this(kinesisClient, streamName, dynamoDBClient, tableName, workerIdentifier, executorService, initialPositionInStream, failoverTimeMillis, epsilonMillis, maxLeasesForWorker, @@ -281,7 +285,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, BillingMode.PROVISIONED); + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, BillingMode.PROVISIONED); } /** @@ -310,7 +314,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param cacheMissWarningModulus * @param initialLeaseTableReadCapacity * @param initialLeaseTableWriteCapacity - * @param deprecatedHierarchicalShardSyncer + * @param hierarchicalShardSyncer * @param tableCreatorCallback * @param dynamoDbRequestTimeout * @param billingMode @@ -326,7 +330,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final int maxListShardsRetryAttempts, final int maxCacheMissesBeforeReload, final long listShardsCacheAllowedAgeInSeconds, final int cacheMissWarningModulus, final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, - final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, + final HierarchicalShardSyncer hierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode) { this(kinesisClient, new StreamConfig(StreamIdentifier.singleStreamInstance(streamName), initialPositionInStream), dynamoDBClient, tableName, @@ -335,7 +339,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { ignoreUnexpectedChildShards, shardSyncIntervalMillis, consistentReads, listShardsBackoffTimeMillis, maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, - deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); + hierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, new DynamoDBLeaseSerializer()); } /** @@ -386,7 +390,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { maxListShardsRetryAttempts, maxCacheMissesBeforeReload, listShardsCacheAllowedAgeInSeconds, cacheMissWarningModulus, initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity, deprecatedHierarchicalShardSyncer, tableCreatorCallback, dynamoDbRequestTimeout, billingMode, leaseSerializer, - null, false); + null, false, LeaseManagementConfig.DEFAULT_LEASE_CLEANUP_CONFIG); this.streamConfig = streamConfig; } @@ -420,6 +424,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { * @param leaseSerializer * @param customShardDetectorProvider * @param isMultiStreamMode + * @param leaseCleanupConfig */ public DynamoDBLeaseManagementFactory(final KinesisAsyncClient kinesisClient, final DynamoDbAsyncClient dynamoDBClient, final String tableName, final String workerIdentifier, @@ -432,7 +437,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { final long initialLeaseTableReadCapacity, final long initialLeaseTableWriteCapacity, final HierarchicalShardSyncer deprecatedHierarchicalShardSyncer, final TableCreatorCallback tableCreatorCallback, Duration dynamoDbRequestTimeout, BillingMode billingMode, LeaseSerializer leaseSerializer, - Function customShardDetectorProvider, boolean isMultiStreamMode) { + Function customShardDetectorProvider, boolean isMultiStreamMode, + LeaseCleanupConfig leaseCleanupConfig) { this.kinesisClient = kinesisClient; this.dynamoDBClient = dynamoDBClient; this.tableName = tableName; @@ -461,6 +467,7 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { this.leaseSerializer = leaseSerializer; this.customShardDetectorProvider = customShardDetectorProvider; this.isMultiStreamMode = isMultiStreamMode; + this.leaseCleanupConfig = leaseCleanupConfig; } @Override @@ -535,4 +542,19 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { maxListShardsRetryAttempts, listShardsCacheAllowedAgeInSeconds, maxCacheMissesBeforeReload, cacheMissWarningModulus, dynamoDbRequestTimeout); } + + /** + * LeaseCleanupManager cleans up leases in the lease table for shards which have either expired past the + * stream's retention period or have been completely processed. + * @param metricsFactory + * @return LeaseCleanupManager + */ + @Override + public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) { + return new LeaseCleanupManager(createLeaseCoordinator(metricsFactory), kinesisClient, + metricsFactory, dynamoDbRequestTimeout, Executors.newSingleThreadScheduledExecutor(), + cleanupLeasesUponShardCompletion, leaseCleanupConfig.leaseCleanupIntervalMillis(), + leaseCleanupConfig.completedLeaseCleanupIntervalMillis(), + leaseCleanupConfig.garbageLeaseCleanupIntervalMillis()); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java new file mode 100644 index 00000000..b840eb09 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java @@ -0,0 +1,35 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases.exceptions; + +import lombok.EqualsAndHashCode; +import lombok.Value; +import lombok.experimental.Accessors; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.ShardInfo; + +/** + * Helper class for cleaning up leases. + */ +@Accessors(fluent = true) +@Value +@EqualsAndHashCode(exclude = {"queueEntryTime"}) +public class LeasePendingDeletion { + private final StreamIdentifier streamIdentifier; + private final Lease lease; + private final ShardInfo shardInfo; +} 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 58e31985..4d894d94 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ConsumerStates.java @@ -146,7 +146,7 @@ class ConsumerStates { @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { - return ShardConsumerState.SHUTDOWN_COMPLETE.consumerState(); + return ShardConsumerState.SHUTTING_DOWN.consumerState(); } @Override @@ -497,7 +497,9 @@ class ConsumerStates { argument.recordsPublisher(), argument.hierarchicalShardSyncer(), argument.metricsFactory(), - input == null ? null : input.childShards()); + input == null ? null : input.childShards(), + argument.streamIdentifier(), + argument.leaseCleanupManager()); } @Override 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 03ddc6ee..0f18891c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShardConsumerArgument.java @@ -22,6 +22,7 @@ import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; @@ -71,4 +72,5 @@ public class ShardConsumerArgument { private final HierarchicalShardSyncer hierarchicalShardSyncer; @NonNull private final MetricsFactory metricsFactory; + private final LeaseCleanupManager leaseCleanupManager; } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 5bd0664f..5f1dcd25 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 @@ -15,6 +15,10 @@ package software.amazon.kinesis.lifecycle; import com.google.common.annotations.VisibleForTesting; + +import java.util.List; +import java.util.Optional; + import lombok.NonNull; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -23,13 +27,16 @@ import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.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.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; @@ -42,7 +49,6 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.util.List; import java.util.Set; import java.util.UUID; import java.util.function.Function; @@ -85,6 +91,10 @@ public class ShutdownTask implements ConsumerTask { private final TaskType taskType = TaskType.SHUTDOWN; private final List childShards; + @NonNull + private final StreamIdentifier streamIdentifier; + @NonNull + private final LeaseCleanupManager leaseCleanupManager; private static final Function leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo); @@ -113,19 +123,26 @@ public class ShutdownTask implements ConsumerTask { // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. // This scenario could happen when customer deletes the stream while leaving the KCL application running. + final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); - updateLeaseWithChildShards(); - } else { - log.warn("Shard {} no longer exists. Shutting down consumer with SHARD_END reason without creating leases for child shards.", leaseKeyProvider.apply(shardInfo)); + updateLeaseWithChildShards(currentShardLease); } - recordProcessorCheckpointer - .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); - recordProcessorCheckpointer.largestPermittedCheckpointValue(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); + final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) + .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); + if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { + recordProcessorCheckpointer + .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); + recordProcessorCheckpointer.largestPermittedCheckpointValue(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); + } + + final LeasePendingDeletion garbageLease = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); + leaseCleanupManager.enqueueForDeletion(garbageLease); } else { throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); } @@ -162,8 +179,8 @@ public class ShutdownTask implements ConsumerTask { if (lastCheckpointValue == null || !lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END)) { throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + leaseKeyProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + - "See ShardRecordProcessor.shardEnded javadocs for more information."); + + leaseKeyProvider.apply(shardInfo) + ". Application must checkpoint upon shard end. " + + "See ShardRecordProcessor.shardEnded javadocs for more information."); } } @@ -189,9 +206,8 @@ public class ShutdownTask implements ConsumerTask { } } - private void updateLeaseWithChildShards() + private void updateLeaseWithChildShards(Lease currentLease) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - final Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); Set childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet()); final Lease updatedLease = currentLease.copy(); @@ -206,7 +222,7 @@ public class ShutdownTask implements ConsumerTask { /* * (non-Javadoc) - * + * * @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerTask#taskType() */ @Override 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 a1601eda..af58d3ab 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 @@ -73,6 +73,7 @@ import software.amazon.kinesis.common.StreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.KinesisClientLibException; import software.amazon.kinesis.exceptions.KinesisClientLibNonRetryableException; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseManagementConfig; @@ -153,6 +154,8 @@ public class SchedulerTest { private WorkerStateChangeListener workerStateChangeListener; @Mock private MultiStreamTracker multiStreamTracker; + @Mock + private LeaseCleanupManager leaseCleanupManager; private Map shardSyncTaskManagerMap; private Map shardDetectorMap; @@ -219,9 +222,9 @@ public class SchedulerTest { final String shardId = "shardId-000000000000"; final String concurrencyToken = "concurrencyToken"; final ShardInfo shardInfo = new ShardInfo(shardId, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory); + final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager); assertNotNull(shardConsumer1); - final ShardConsumer shardConsumer2 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory); + final ShardConsumer shardConsumer2 = scheduler.createOrGetShardConsumer(shardInfo, shardRecordProcessorFactory, leaseCleanupManager); assertNotNull(shardConsumer2); assertSame(shardConsumer1, shardConsumer2); @@ -229,7 +232,7 @@ public class SchedulerTest { final String anotherConcurrencyToken = "anotherConcurrencyToken"; final ShardInfo shardInfo2 = new ShardInfo(shardId, anotherConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - final ShardConsumer shardConsumer3 = scheduler.createOrGetShardConsumer(shardInfo2, shardRecordProcessorFactory); + final ShardConsumer shardConsumer3 = scheduler.createOrGetShardConsumer(shardInfo2, shardRecordProcessorFactory, leaseCleanupManager); assertNotNull(shardConsumer3); assertNotSame(shardConsumer1, shardConsumer3); @@ -261,9 +264,9 @@ public class SchedulerTest { schedulerSpy.runProcessLoop(); schedulerSpy.runProcessLoop(); - verify(schedulerSpy).buildConsumer(same(initialShardInfo.get(0)), eq(shardRecordProcessorFactory)); - verify(schedulerSpy, never()).buildConsumer(same(firstShardInfo.get(0)), eq(shardRecordProcessorFactory)); - verify(schedulerSpy, never()).buildConsumer(same(secondShardInfo.get(0)), eq(shardRecordProcessorFactory)); + verify(schedulerSpy).buildConsumer(same(initialShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)); + verify(schedulerSpy, never()).buildConsumer(same(firstShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)); + verify(schedulerSpy, never()).buildConsumer(same(secondShardInfo.get(0)), eq(shardRecordProcessorFactory), eq(leaseCleanupManager)); verify(checkpoint).getCheckpointObject(eq(shardId)); } @@ -279,10 +282,10 @@ public class SchedulerTest { ExtendedSequenceNumber.TRIM_HORIZON); final ShardInfo shardInfo1 = new ShardInfo(shard1, concurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - final ShardConsumer shardConsumer0 = scheduler.createOrGetShardConsumer(shardInfo0, shardRecordProcessorFactory); + final ShardConsumer shardConsumer0 = scheduler.createOrGetShardConsumer(shardInfo0, shardRecordProcessorFactory, leaseCleanupManager); final ShardConsumer shardConsumer0WithAnotherConcurrencyToken = - scheduler.createOrGetShardConsumer(shardInfo0WithAnotherConcurrencyToken, shardRecordProcessorFactory); - final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo1, shardRecordProcessorFactory); + scheduler.createOrGetShardConsumer(shardInfo0WithAnotherConcurrencyToken, shardRecordProcessorFactory, leaseCleanupManager); + final ShardConsumer shardConsumer1 = scheduler.createOrGetShardConsumer(shardInfo1, shardRecordProcessorFactory, leaseCleanupManager); Set shards = new HashSet<>(); shards.add(shardInfo0); @@ -397,11 +400,11 @@ public class SchedulerTest { schedulerSpy.runProcessLoop(); initialShardInfo.stream().forEach( - shardInfo -> verify(schedulerSpy).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory))); + shardInfo -> verify(schedulerSpy).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), same(leaseCleanupManager))); firstShardInfo.stream().forEach( - shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory))); + shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager))); secondShardInfo.stream().forEach( - shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory))); + shardInfo -> verify(schedulerSpy, never()).buildConsumer(same(shardInfo), eq(shardRecordProcessorFactory), eq(leaseCleanupManager))); } @@ -1071,6 +1074,11 @@ public class SchedulerTest { public ShardDetector createShardDetector(StreamConfig streamConfig) { return shardDetectorMap.get(streamConfig.streamIdentifier()); } + + @Override + public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) { + return leaseCleanupManager; + } } private class TestKinesisCheckpointFactory implements CheckpointFactory { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java new file mode 100644 index 00000000..eb06a4a0 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseCleanupManagerTest.java @@ -0,0 +1,347 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.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.ResourceNotFoundException; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; +import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.NullMetricsFactory; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class LeaseCleanupManagerTest { + + private ShardInfo shardInfo; + private StreamIdentifier streamIdentifier; + private String concurrencyToken = "1234"; + + private String shardId = "shardId"; + private String splitParent = "splitParent"; + private String mergeParent1 = "mergeParent-1"; + private String mergeParent2 = "mergeParent-2"; + + private Duration maxFutureWait = Duration.ofSeconds(1); + private long leaseCleanupIntervalMillis = Duration.ofSeconds(1).toMillis(); + private long completedLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); + private long garbageLeaseCleanupIntervalMillis = Duration.ofSeconds(0).toMillis(); + private boolean cleanupLeasesOfCompletedShards = true; + private LeaseCleanupManager leaseCleanupManager; + private static final MetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory(); + + @Mock + private LeaseRefresher leaseRefresher; + @Mock + private LeaseCoordinator leaseCoordinator; + @Mock + private KinesisAsyncClient kinesis; + @Mock + private ScheduledExecutorService deletionThreadPool; + + @Before + public void setUp() throws Exception { + shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + streamIdentifier = StreamIdentifier.singleStreamInstance("streamName"); + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, + NULL_METRICS_FACTORY, maxFutureWait, deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, + completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); + + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(any(Lease.class), any(UUID.class), any(String.class), any(String.class))).thenReturn(true); + } + + /** + * Tests that when both child shard leases are present, we are able to delete the parent shard for the completed + * shard case. + */ + @Test + public final void testParentShardLeaseDeletedSplitCase() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 1); + } + + /** + * Tests that when both child shard leases are present, we are able to delete the parent shard for the completed + * shard case. + */ + @Test + public final void testParentShardLeaseDeletedMergeCase() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 1); + } + + /** + * Tests that if cleanupLeasesOfCompletedShards is not enabled by the customer, then no leases are cleaned up for + * the completed shard case. + */ + @Test + public final void testNoLeasesDeletedWhenNotEnabled() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + cleanupLeasesOfCompletedShards = false; + + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, NULL_METRICS_FACTORY, maxFutureWait, + deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + garbageLeaseCleanupIntervalMillis); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 0); + } + + /** + * Tests that if some of the child shard leases are missing, we fail fast and don't delete the parent shard lease + * for the completed shard case. + */ + @Test + public final void testNoCleanupWhenSomeChildShardLeasesAreNotPresent() throws Exception { + List childShards = childShardsForSplit(); + + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, ExtendedSequenceNumber.LATEST, false, 0); + } + + /** + * Tests that if some child shard leases haven't begun processing (at least one lease w/ checkpoint TRIM_HORIZON), + * we don't delete them for the completed shard case. + */ + @Test + public final void testParentShardLeaseNotDeletedWhenChildIsAtTrim() throws Exception { + testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.TRIM_HORIZON); + } + + /** + * Tests that if some child shard leases haven't begun processing (at least one lease w/ checkpoint AT_TIMESTAMP), + * we don't delete them for the completed shard case. + */ + @Test + public final void testParentShardLeaseNotDeletedWhenChildIsAtTimestamp() throws Exception { + testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber.AT_TIMESTAMP); + } + + private final void testParentShardLeaseNotDeletedWhenChildIsAtPosition(ExtendedSequenceNumber extendedSequenceNumber) + throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), extendedSequenceNumber, 0); + } + + /** + * Tests that if a lease's parents are still present, we do not delete the lease. + */ + @Test + public final void testLeaseNotDeletedWhenParentsStillPresent() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.singleton("parent"), + ExtendedSequenceNumber.LATEST); + + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForMerge(), ExtendedSequenceNumber.LATEST, 0); + } + + /** + * Tests ResourceNotFound case for if a shard expires, that we delete the lease when shardExpired is found. + */ + @Test + public final void testLeaseDeletedWhenShardDoesNotExist() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + + testLeaseDeletedWhenShardDoesNotExist(heldLease); + } + + /** + * Tests ResourceNotFound case when completed lease cleanup is disabled. + * @throws Exception + */ + @Test + public final void testLeaseDeletedWhenShardDoesNotExistAndCleanupCompletedLeaseDisabled() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + + cleanupLeasesOfCompletedShards = false; + + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, NULL_METRICS_FACTORY, maxFutureWait, + deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + garbageLeaseCleanupIntervalMillis); + + testLeaseDeletedWhenShardDoesNotExist(heldLease); + } + + public void testLeaseDeletedWhenShardDoesNotExist(Lease heldLease) throws Exception { + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); + when(kinesis.getShardIterator(any(GetShardIteratorRequest.class))).thenThrow(ResourceNotFoundException.class); + when(leaseRefresher.getLease(heldLease.leaseKey())).thenReturn(heldLease); + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); + leaseCleanupManager.cleanupLeases(); + + verify(leaseRefresher, times(1)).deleteLease(heldLease); + } + + /** + * Tests that if a lease deletion fails, it's re-enqueued for deletion. + */ + @Test + public final void testFailedDeletionsReEnqueued() throws Exception { + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + + final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); + when(kinesis.getShardIterator(any(GetShardIteratorRequest.class))).thenThrow(Exception.class); + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); + Assert.assertEquals(1, leaseCleanupManager.leasesPendingDeletion()); + } + + /** + * Tests duplicate leases are not enqueued for deletion. + */ + @Test + public final void testNoDuplicateLeasesEnqueued() { + // Disable lease cleanup so that the queue isn't drained while the test is running. + cleanupLeasesOfCompletedShards = false; + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, + NULL_METRICS_FACTORY, maxFutureWait, deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, + completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); + shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); + + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); + + // Enqueue the same lease twice. + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); + Assert.assertEquals(1, leaseCleanupManager.leasesPendingDeletion()); + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); + Assert.assertEquals(1, leaseCleanupManager.leasesPendingDeletion()); + } + + private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + ExtendedSequenceNumber extendedSequenceNumber, + int expectedDeletedLeases) throws Exception { + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, extendedSequenceNumber, true, expectedDeletedLeases); + } + + private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + ExtendedSequenceNumber extendedSequenceNumber, + boolean childShardLeasesPresent, + int expectedDeletedLeases) throws Exception { + + final Lease lease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", shardInfo.parentShardIds(), + childShards.stream().map(c -> c.shardId()).collect(Collectors.toSet())); + final List childShardLeases = childShards.stream().map(c -> LeaseHelper.createLease( + ShardInfo.getLeaseKey(shardInfo, c.shardId()), "leaseOwner", Collections.singleton(shardInfo.shardId()), + Collections.emptyList(), extendedSequenceNumber)).collect(Collectors.toList()); + + final List parentShardLeases = lease.parentShardIds().stream().map(p -> + LeaseHelper.createLease(ShardInfo.getLeaseKey(shardInfo, p), "leaseOwner", Collections.emptyList(), + Collections.singleton(shardInfo.shardId()), extendedSequenceNumber)).collect(Collectors.toList()); + + when(leaseRefresher.getLease(lease.leaseKey())).thenReturn(lease); + for (Lease parentShardLease : parentShardLeases) { + when(leaseRefresher.getLease(parentShardLease.leaseKey())).thenReturn(parentShardLease); + } + if (childShardLeasesPresent) { + for (Lease childShardLease : childShardLeases) { + when(leaseRefresher.getLease(childShardLease.leaseKey())).thenReturn(childShardLease); + } + } + + GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder() + .shardIterator("123") + .build(); + when(kinesis.getShardIterator(any(GetShardIteratorRequest.class))).thenReturn(CompletableFuture.completedFuture(getShardIteratorResponse)); + + GetRecordsResponse getRecordsResponse = GetRecordsResponse.builder() + .records(Collections.emptyList()) + .childShards(childShards) + .build(); + when(kinesis.getRecords(any(GetRecordsRequest.class))).thenReturn(CompletableFuture.completedFuture(getRecordsResponse)); + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, lease, shardInfo)); + leaseCleanupManager.cleanupLeases(); + + verify(leaseRefresher, times(expectedDeletedLeases)).deleteLease(any(Lease.class)); + } + + private List childShardsForSplit() { + List parentShards = Arrays.asList(splitParent); + + ChildShard leftChild = ChildShard.builder() + .shardId("leftChild") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + ChildShard rightChild = ChildShard.builder() + .shardId("rightChild") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")) + .build(); + + return Arrays.asList(leftChild, rightChild); + } + + private List childShardsForMerge() { + List parentShards = Arrays.asList(mergeParent1, mergeParent2); + + ChildShard child = ChildShard.builder() + .shardId("onlyChild") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "99")) + .build(); + + return Collections.singletonList(child); + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java new file mode 100644 index 00000000..0e10bc48 --- /dev/null +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/LeaseHelper.java @@ -0,0 +1,44 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package software.amazon.kinesis.leases; + +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.Collection; +import java.util.Collections; + +public class LeaseHelper { + + public static Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + return createLease(leaseKey, leaseOwner, parentShardIds, Collections.emptySet(), ExtendedSequenceNumber.LATEST); + } + + public static Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds, Collection childShardIds) { + return createLease(leaseKey, leaseOwner, parentShardIds, childShardIds, ExtendedSequenceNumber.LATEST); + } + + public static Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds, + Collection childShardIds, ExtendedSequenceNumber extendedSequenceNumber) { + Lease lease = new Lease(); + lease.leaseKey(leaseKey); + lease.leaseOwner(leaseOwner); + lease.parentShardIds(parentShardIds); + lease.childShardIds(childShardIds); + lease.checkpoint(extendedSequenceNumber); + + return lease; + } +} diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java index 06e5afc7..235937d0 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ConsumerStatesTest.java @@ -46,6 +46,7 @@ import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; @@ -102,6 +103,8 @@ public class ConsumerStatesTest { private ProcessRecordsInput processRecordsInput; @Mock private TaskExecutionListener taskExecutionListener; + @Mock + private LeaseCleanupManager leaseCleanupManager; private long parentShardPollIntervalMillis = 0xCAFE; private boolean cleanupLeasesOfCompletedShards = true; @@ -122,7 +125,7 @@ public class ConsumerStatesTest { taskBackoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, listShardsBackoffTimeInMillis, maxListShardsRetryAttempts, shouldCallProcessRecordsEvenForEmptyRecordList, idleTimeInMillis, INITIAL_POSITION_IN_STREAM, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, shardDetector, - new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory); + new AggregatorUtil(), hierarchicalShardSyncer, metricsFactory, leaseCleanupManager); when(shardInfo.shardId()).thenReturn("shardId-000000000000"); when(shardInfo.streamIdentifierSerOpt()).thenReturn(Optional.of(StreamIdentifier.singleStreamInstance(STREAM_NAME).serialize())); consumer = spy(new ShardConsumer(recordsPublisher, executorService, shardInfo, logWarningForTaskAfterMillis, @@ -148,7 +151,7 @@ public class ConsumerStatesTest { assertThat(state.successTransition(), equalTo(ShardConsumerState.INITIALIZING.consumerState())); for (ShutdownReason shutdownReason : ShutdownReason.values()) { assertThat(state.shutdownTransition(shutdownReason), - equalTo(ShardConsumerState.SHUTDOWN_COMPLETE.consumerState())); + equalTo(ShardConsumerState.SHUTTING_DOWN.consumerState())); } assertThat(state.state(), equalTo(ShardConsumerState.WAITING_ON_PARENT_SHARDS)); 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 f65655db..5920646c 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 @@ -19,7 +19,6 @@ 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.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -27,8 +26,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.UUID; @@ -41,8 +38,6 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.awssdk.services.kinesis.model.ChildShard; -import software.amazon.awssdk.services.kinesis.model.SequenceNumberRange; -import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -50,13 +45,16 @@ import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; +import software.amazon.kinesis.leases.LeaseHelper; import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardObjectHelper; import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; import software.amazon.kinesis.leases.exceptions.DependencyException; +import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.events.LeaseLostInput; @@ -86,6 +84,7 @@ public class ShutdownTaskTest { private boolean ignoreUnexpectedChildShards = false; private ShardInfo shardInfo; private ShutdownTask task; + private StreamIdentifier streamIdentifier = StreamIdentifier.singleStreamInstance("streamName"); @Mock private RecordsPublisher recordsPublisher; @@ -103,6 +102,8 @@ public class ShutdownTaskTest { private HierarchicalShardSyncer hierarchicalShardSyncer; @Mock private ShardRecordProcessor shardRecordProcessor; + @Mock + private LeaseCleanupManager leaseCleanupManager; @Before public void setUp() throws Exception { @@ -119,7 +120,7 @@ public class ShutdownTaskTest { task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards()); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards(), streamIdentifier, leaseCleanupManager); } /** @@ -129,8 +130,9 @@ public class ShutdownTaskTest { @Test public final void testCallWhenApplicationDoesNotCheckpoint() throws Exception { when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); - Lease heldLease = createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId")); + 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); @@ -169,13 +171,14 @@ public class ShutdownTaskTest { task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards()); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShards(), streamIdentifier, leaseCleanupManager); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - Lease heldLease = createLease("shardId-0", "leaseOwner", Collections.singleton("parentShardId")); + 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()); @@ -185,6 +188,7 @@ public class ShutdownTaskTest { verify(leaseCoordinator).updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); 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)); } /** @@ -192,23 +196,24 @@ public class ShutdownTaskTest { * This test is for the scenario that a ShutdownTask is created for detecting a false Shard End. */ @Test - public final void testCallWhenShardNotFound() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + 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<>()); + hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>(), streamIdentifier, leaseCleanupManager); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseRefresher.getLease("shardId-4")).thenReturn(heldLease); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-4")).thenReturn(heldLease); final TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); - verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); - verify(leaseCoordinator, never()).getCurrentlyHeldLease(shardInfo.shardId()); verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); } @@ -220,11 +225,11 @@ public class ShutdownTaskTest { @Test public final void testCallWhenLeaseLost() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-4", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); + ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, - LEASE_LOST_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, - ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, - hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>()); + LEASE_LOST_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, new ArrayList<>(), streamIdentifier, leaseCleanupManager); final TaskResult result = task.call(); assertNull(result.getException()); @@ -232,10 +237,9 @@ public class ShutdownTaskTest { verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); verify(leaseCoordinator, never()).getAssignments(); - verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); - verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(any(Lease.class)); } - /** * Test method for {@link ShutdownTask#taskType()}. */ @@ -262,14 +266,4 @@ public class ShutdownTaskTest { childShards.add(rightChild); return childShards; } - - private Lease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { - Lease lease = new Lease(); - lease.leaseKey(leaseKey); - lease.leaseOwner(leaseOwner); - lease.parentShardIds(parentShardIds); - - return lease; - } - } From dc078ead39bdda1ef4795e86348fde5f6af195b6 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 10 Jun 2020 19:49:06 -0400 Subject: [PATCH 104/159] Reverting pom change --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4e8da722..f21b230e 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.10.65-SNAPSHOT + 2.11.8-SNAPSHOT From 445dd106b2996de1f16b54c3d577754245b17f42 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Thu, 11 Jun 2020 19:48:24 -0400 Subject: [PATCH 105/159] Adding logging --- .../amazon/kinesis/leases/LeaseCleanupManager.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 ba108748..6d6b46a8 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 @@ -157,7 +157,12 @@ public class LeaseCleanupManager { if (CollectionUtils.isNullOrEmpty(childShardKeys)) { try { childShardKeys = getChildShardsFromService(shardInfo, streamIdentifier); - updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + + if (childShardKeys == null) { + log.error("No child shards returned from service for shard {} for {}.", shardInfo.shardId(), streamIdentifier.streamName()); + } else { + updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + } } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } finally { From c8422745d9cb98033723b7aba06ec0991b84ee9c Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+jushkem@users.noreply.github.com> Date: Thu, 11 Jun 2020 19:56:04 -0400 Subject: [PATCH 106/159] Removing lease cleanup from HierarchicalShardSyncer (#30) * Removing lease cleanup from HierarchicalShardSyncer --- .../leases/HierarchicalShardSyncer.java | 230 +--------------- .../amazon/kinesis/leases/ShardSyncTask.java | 2 +- .../leases/exceptions/ShardSyncer.java | 8 +- .../leases/HierarchicalShardSyncerTest.java | 248 ++---------------- 4 files changed, 34 insertions(+), 454 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 68a1701d..5c18ee85 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,15 +17,11 @@ package software.amazon.kinesis.leases; import java.io.Serializable; import java.math.BigInteger; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.function.BiFunction; @@ -106,9 +102,7 @@ public class HierarchicalShardSyncer { * @param leaseRefresher * @param initialPosition * @param scope - * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards - * @param garbageCollectLeases * @throws DependencyException * @throws InvalidStateException * @throws ProvisionedThroughputException @@ -117,20 +111,18 @@ public class HierarchicalShardSyncer { // CHECKSTYLE:OFF CyclomaticComplexity public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final MetricsScope scope, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, - final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) + final MetricsScope scope, final boolean ignoreUnexpectedChildShards, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); - checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, scope, garbageCollectLeases, + checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, ignoreUnexpectedChildShards, scope, isLeaseTableEmpty); } //Provide a pre-collcted list of shards to avoid calling ListShards API public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - List latestShards, final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, - final MetricsScope scope, final boolean garbageCollectLeases, final boolean isLeaseTableEmpty) + 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 @@ -166,13 +158,6 @@ public class HierarchicalShardSyncer { } final List trackedLeases = new ArrayList<>(currentLeases); trackedLeases.addAll(newLeasesToCreate); - if (!isLeaseTableEmpty && garbageCollectLeases) { - cleanupGarbageLeases(shardDetector, latestShards, trackedLeases, leaseRefresher, multiStreamArgs); - } - if (!isLeaseTableEmpty && cleanupLeasesOfCompletedShards) { - cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases, - leaseRefresher, multiStreamArgs); - } } /** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls @@ -205,44 +190,6 @@ public class HierarchicalShardSyncer { .flatMap(entry -> shardIdToChildShardIdsMap.get(entry.getKey()).stream()).collect(Collectors.toSet()); } - /** - * Note: this has package level access for testing purposes. - * Useful for asserting that we don't have an incomplete shard list following a reshard operation. - * We verify that if the shard is present in the shard list, it is closed and its hash key range - * is covered by its child shards. - * @param shardIdsOfClosedShards Id of the shard which is expected to be closed - * @return ShardIds of child shards (children of the expectedClosedShard) - * @throws KinesisClientLibIOException - */ - synchronized void assertClosedShardsAreCoveredOrAbsent(final Map shardIdToShardMap, - final Map> shardIdToChildShardIdsMap, final Set shardIdsOfClosedShards) - throws KinesisClientLibIOException { - final String exceptionMessageSuffix = "This can happen if we constructed the list of shards " - + " while a reshard operation was in progress."; - - for (String shardId : shardIdsOfClosedShards) { - final Shard shard = shardIdToShardMap.get(shardId); - if (shard == null) { - log.info("{} : Shard {} is not present in Kinesis anymore.", streamIdentifier, shardId); - continue; - } - - final String endingSequenceNumber = shard.sequenceNumberRange().endingSequenceNumber(); - if (endingSequenceNumber == null) { - throw new KinesisClientLibIOException("Shard " + shardIdsOfClosedShards - + " is not closed. " + exceptionMessageSuffix); - } - - final Set childShardIds = shardIdToChildShardIdsMap.get(shardId); - if (childShardIds == null) { - throw new KinesisClientLibIOException("Incomplete shard list: Closed shard " + shardId - + " has no children." + exceptionMessageSuffix); - } - - assertHashRangeOfClosedShardIsCovered(shard, shardIdToShardMap, childShardIds); - } - } - private synchronized void assertHashRangeOfClosedShardIsCovered(final Shard closedShard, final Map shardIdToShardMap, final Set childShardIds) throws KinesisClientLibIOException { @@ -617,177 +564,6 @@ public class HierarchicalShardSyncer { return parentShardIds; } - /** - * Delete leases corresponding to shards that no longer exist in the stream. Current scheme: Delete a lease if: - *
    - *
  • The corresponding shard is not present in the list of Kinesis shards
  • - *
  • The parentShardIds listed in the lease are also not present in the list of Kinesis shards.
  • - *
- * - * @param shards - * List of all Kinesis shards (assumed to be a consistent snapshot - when stream is in Active state). - * @param trackedLeases - * List of - * @param leaseRefresher - * @throws KinesisClientLibIOException - * Thrown if we couldn't get a fresh shard list from Kinesis. - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - */ - private static void cleanupGarbageLeases(@NonNull final ShardDetector shardDetector, final List shards, - final List trackedLeases, final LeaseRefresher leaseRefresher, - final MultiStreamArgs multiStreamArgs) throws KinesisClientLibIOException, - DependencyException, InvalidStateException, ProvisionedThroughputException { - final String streamIdentifier = getStreamIdentifier(multiStreamArgs); - final Set kinesisShards = shards.stream().map(Shard::shardId).collect(Collectors.toSet()); - - // Check if there are leases for non-existent shards - final List garbageLeases = trackedLeases.stream() - .filter(lease -> isCandidateForCleanup(lease, kinesisShards, multiStreamArgs)).collect(Collectors.toList()); - - if (!CollectionUtils.isNullOrEmpty(garbageLeases)) { - log.info("{} : Found {} candidate leases for cleanup. Refreshing list of" - + " Kinesis shards to pick up recent/latest shards", streamIdentifier, garbageLeases.size()); - final Set currentKinesisShardIds = getShardList(shardDetector).stream().map(Shard::shardId) - .collect(Collectors.toSet()); - - for (Lease lease : garbageLeases) { - if (isCandidateForCleanup(lease, currentKinesisShardIds, multiStreamArgs)) { - log.info("{} : Deleting lease for shard {} as it is not present in Kinesis stream.", - streamIdentifier, lease.leaseKey()); - leaseRefresher.deleteLease(lease); - } - } - } - } - - /** - * Note: This method has package level access, solely for testing purposes. - * - * @param lease Candidate shard we are considering for deletion. - * @param currentKinesisShardIds - * @return true if neither the shard (corresponding to the lease), nor its parents are present in - * currentKinesisShardIds - * @throws KinesisClientLibIOException Thrown if currentKinesisShardIds contains a parent shard but not the child - * shard (we are evaluating for deletion). - */ - static boolean isCandidateForCleanup(final Lease lease, final Set currentKinesisShardIds, - final MultiStreamArgs multiStreamArgs) - throws KinesisClientLibIOException { - - final String streamIdentifier = getStreamIdentifier(multiStreamArgs); - - boolean isCandidateForCleanup = true; - final String shardId = shardIdFromLeaseDeducer.apply(lease, multiStreamArgs); - - if (currentKinesisShardIds.contains(shardId)) { - isCandidateForCleanup = false; - } else { - log.info("{} : Found lease for non-existent shard: {}. Checking its parent shards", streamIdentifier, shardId); - final Set parentShardIds = lease.parentShardIds(); - for (String parentShardId : parentShardIds) { - - // Throw an exception if the parent shard exists (but the child does not). - // This may be a (rare) race condition between fetching the shard list and Kinesis expiring shards. - if (currentKinesisShardIds.contains(parentShardId)) { - final String message = String.format("Parent shard %s exists but not the child shard %s", - parentShardId, shardId); - log.info("{} : {}", streamIdentifier, message); - throw new KinesisClientLibIOException(message); - } - } - } - - return isCandidateForCleanup; - } - - /** - * Private helper method. - * Clean up leases for shards that meet the following criteria: - * a/ the shard has been fully processed (checkpoint is set to SHARD_END) - * b/ we've begun processing all the child shards: we have leases for all child shards and their checkpoint is not - * TRIM_HORIZON. - * - * @param currentLeases List of leases we evaluate for clean up - * @param shardIdToShardMap Map of shardId->Shard (assumed to include all Kinesis shards) - * @param shardIdToChildShardIdsMap Map of shardId->childShardIds (assumed to include all Kinesis shards) - * @param trackedLeases List of all leases we are tracking. - * @param leaseRefresher Lease refresher (will be used to delete leases) - * @throws DependencyException - * @throws InvalidStateException - * @throws ProvisionedThroughputException - * @throws KinesisClientLibIOException - */ - private synchronized void cleanupLeasesOfFinishedShards(final Collection currentLeases, - final Map shardIdToShardMap, final Map> shardIdToChildShardIdsMap, - final List trackedLeases, final LeaseRefresher leaseRefresher, - final MultiStreamArgs multiStreamArgs) throws DependencyException, - InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - final List leasesOfClosedShards = currentLeases.stream() - .filter(lease -> lease.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) - .collect(Collectors.toList()); - final Set shardIdsOfClosedShards = leasesOfClosedShards.stream() - .map(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs)).collect(Collectors.toSet()); - - if (!CollectionUtils.isNullOrEmpty(leasesOfClosedShards)) { - assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, shardIdsOfClosedShards); - //TODO: Verify before LTR launch that ending sequence number is still returned from the service. - Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMap, multiStreamArgs); - leasesOfClosedShards.sort(startingSequenceNumberComparator); - final Map trackedLeaseMap = trackedLeases.stream() - .collect(Collectors.toMap(lease -> shardIdFromLeaseDeducer.apply(lease, multiStreamArgs), Function.identity())); - - for (Lease leaseOfClosedShard : leasesOfClosedShards) { - final String closedShardId = shardIdFromLeaseDeducer.apply(leaseOfClosedShard, multiStreamArgs); - final Set childShardIds = shardIdToChildShardIdsMap.get(closedShardId); - if (closedShardId != null && !CollectionUtils.isNullOrEmpty(childShardIds)) { - cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseRefresher, multiStreamArgs); - } - } - } - } - - /** - * Delete lease for the closed shard. Rules for deletion are: - * a/ the checkpoint for the closed shard is SHARD_END, - * b/ there are leases for all the childShardIds and their checkpoint is NOT TRIM_HORIZON - * Note: This method has package level access solely for testing purposes. - * - * @param closedShardId Identifies the closed shard - * @param childShardIds ShardIds of children of the closed shard - * @param trackedLeases shardId->Lease map with all leases we are tracking (should not be null) - * @param leaseRefresher - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - */ - synchronized void cleanupLeaseForClosedShard(final String closedShardId, final Set childShardIds, - final Map trackedLeases, final LeaseRefresher leaseRefresher, final MultiStreamArgs multiStreamArgs) - throws DependencyException, InvalidStateException, ProvisionedThroughputException { - final Lease leaseForClosedShard = trackedLeases.get(closedShardId); - final List childShardLeases = childShardIds.stream().map(trackedLeases::get).filter(Objects::nonNull) - .collect(Collectors.toList()); - - if (leaseForClosedShard != null && leaseForClosedShard.checkpoint().equals(ExtendedSequenceNumber.SHARD_END) - && childShardLeases.size() == childShardIds.size()) { - boolean okayToDelete = true; - for (Lease lease : childShardLeases) { - if (lease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON)) { - okayToDelete = false; - break; - } - } - - if (okayToDelete) { - log.info("{} : Deleting lease for shard {} as it has been completely processed and processing of child " - + "shards has begun.", streamIdentifier, shardIdFromLeaseDeducer.apply(leaseForClosedShard, multiStreamArgs)); - leaseRefresher.deleteLease(leaseForClosedShard); - } - } - } - public synchronized Lease createLeaseForChildShard(final ChildShard childShard, final StreamIdentifier streamIdentifier) throws InvalidStateException { final MultiStreamArgs multiStreamArgs = new MultiStreamArgs(isMultiStreamMode, streamIdentifier); 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 f5c7ab8a..820d4528 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 @@ -68,7 +68,7 @@ public class ShardSyncTask implements ConsumerTask { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, - initialPosition, scope, garbageCollectLeases, ignoreUnexpectedChildShards, cleanupLeasesUponShardCompletion, + initialPosition, scope, ignoreUnexpectedChildShards, leaseRefresher.isLeaseTableEmpty()); if (shardSyncTaskIdleTimeMillis > 0) { 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 182854ff..f7ec12c5 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/ShardSyncer.java @@ -27,7 +27,6 @@ public class ShardSyncer { * @param shardDetector * @param leaseRefresher * @param initialPosition - * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards * @param scope * @throws DependencyException @@ -38,10 +37,9 @@ public class ShardSyncer { @Deprecated public static synchronized void checkAndCreateLeasesForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, - final boolean cleanupLeasesOfCompletedShards, final boolean ignoreUnexpectedChildShards, - final MetricsScope scope) throws DependencyException, InvalidStateException, ProvisionedThroughputException, - KinesisClientLibIOException, InterruptedException { + final boolean ignoreUnexpectedChildShards, final MetricsScope scope) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { HIERARCHICAL_SHARD_SYNCER.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, - scope, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, leaseRefresher.isLeaseTableEmpty()); + scope, ignoreUnexpectedChildShards, leaseRefresher.isLeaseTableEmpty()); } } 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 1be28b1d..c390987c 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -89,8 +89,6 @@ public class HierarchicalShardSyncerTest { private static final HierarchicalShardSyncer.MultiStreamArgs MULTI_STREAM_ARGS = new HierarchicalShardSyncer.MultiStreamArgs( MULTISTREAM_MODE_ON, StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER)); - private final boolean garbageCollectLeases = true; - private final boolean cleanupLeasesOfCompletedShards = true; private final boolean ignoreUnexpectedChildShards = false; private HierarchicalShardSyncer hierarchicalShardSyncer; @@ -302,7 +300,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10")); @@ -337,7 +335,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -380,7 +378,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - latestShards, cleanupLeasesOfCompletedShards, false, SCOPE, garbageCollectLeases, + latestShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( @@ -419,7 +417,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - latestShards, cleanupLeasesOfCompletedShards, false, SCOPE, garbageCollectLeases, + latestShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( @@ -456,7 +454,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - new ArrayList(), cleanupLeasesOfCompletedShards, false, SCOPE, garbageCollectLeases, + new ArrayList(), false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>(); @@ -663,7 +661,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -683,7 +681,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, + INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); @@ -719,7 +717,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, true, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -764,7 +762,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, true, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -820,7 +818,7 @@ public class HierarchicalShardSyncerTest { // Initial call: No leases present, create leases. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -833,126 +831,19 @@ public class HierarchicalShardSyncerTest { verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); - // Second call: Leases present, with shardId-0 being at ShardEnd causing cleanup. + // Second call: Leases present, no leases should be deleted. hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List deleteLeases = leaseDeleteCaptor.getAllValues(); final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) .collect(Collectors.toSet()); - final Set expectedShardIds = new HashSet<>(Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); + assertThat(deleteLeases.size(), equalTo(0)); verify(shardDetector, times(2)).listShards(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(2)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); - } - - @Test(expected = DependencyException.class) - public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithDeleteLeaseExceptions() - throws Exception { - testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions(ExtendedSequenceNumber.TRIM_HORIZON, - INITIAL_POSITION_TRIM_HORIZON); - } - - @Test(expected = DependencyException.class) - public void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithDeleteLeaseExceptions() - throws Exception { - testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions(ExtendedSequenceNumber.AT_TIMESTAMP, - INITIAL_POSITION_AT_TIMESTAMP); - } - - private void testCheckAndCreateLeasesForNewShardsAndClosedShardWithDeleteLeaseExceptions( - final ExtendedSequenceNumber sequenceNumber, - final InitialPositionInStreamExtended position) - throws Exception { - final String shardIdPrefix = "shardId-%d"; - final List shards = constructShardListForGraphA(); - final List leases = createLeasesFromShards(shards, sequenceNumber, LEASE_OWNER); - - // Marking shardId-0 as ShardEnd. - leases.stream().filter(lease -> String.format(shardIdPrefix, 0).equals(lease.leaseKey())).findFirst() - .ifPresent(lease -> lease.checkpoint(ExtendedSequenceNumber.SHARD_END)); - - // Marking child of shardId-0 to be processed and not at TRIM_HORIZON. - leases.stream().filter(lease -> String.format(shardIdPrefix, 6).equals(lease.leaseKey())).findFirst() - .ifPresent(lease -> lease.checkpoint(new ExtendedSequenceNumber("1"))); - - final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()).thenReturn(leases); - when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())).thenReturn(true); - doThrow(new DependencyException(new Throwable("Throw for DeleteLease"))).doNothing() - .when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); - - // Initial call: Call to create leases. - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); - - final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); - - assertThat(createLeases, equalTo(expectedCreateLeases)); - - verify(shardDetector, times(1)).listShards(); - verify(dynamoDBLeaseRefresher, times(1)).listLeases(); - verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); - - try { - // Second call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete fails. - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - } finally { - List deleteLeases = leaseDeleteCaptor.getAllValues(); - Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - final Set expectedShardIds = new HashSet<>( - Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); - - verify(shardDetector, times(2)).listShards(); - verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, times(2)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); - - // Final call: Leases already present. ShardId-0 is at ShardEnd and needs to be cleaned up. Delete passes. - hierarchicalShardSyncer - .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - deleteLeases = leaseDeleteCaptor.getAllValues(); - - shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint).collect(Collectors.toSet()); - - assertThat(deleteLeases.size(), equalTo(2)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); - - verify(shardDetector, times(3)).listShards(); - verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); - verify(dynamoDBLeaseRefresher, times(3)).listLeases(); - verify(dynamoDBLeaseRefresher, times(2)).deleteLease(any(Lease.class)); - } } @Test(expected = DependencyException.class) @@ -985,20 +876,18 @@ public class HierarchicalShardSyncerTest { .ifPresent(lease -> lease.checkpoint(new ExtendedSequenceNumber("1"))); final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()) .thenThrow(new DependencyException(new Throwable("Throw for ListLeases"))) .thenReturn(Collections.emptyList()).thenReturn(leases); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())).thenReturn(true); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); try { // Initial call: Call to create leases. Fails on ListLeases hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -1008,7 +897,7 @@ public class HierarchicalShardSyncerTest { // Second call: Leases not present, leases will be created. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); final Set expectedCreateLeases = getExpectedLeasesForGraphA(shards, sequenceNumber, position); @@ -1023,26 +912,17 @@ public class HierarchicalShardSyncerTest { // Final call: Leases present, belongs to TestOwner, shardId-0 is at ShardEnd should be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, 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()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( Collections.singletonList(String.format(shardIdPrefix, 0))); final Set expectedSequenceNumbers = new HashSet<>( Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); - verify(shardDetector, times(3)).listShards(); verify(dynamoDBLeaseRefresher, times(expectedCreateLeases.size())).createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } } @@ -1076,20 +956,18 @@ public class HierarchicalShardSyncerTest { .ifPresent(lease -> lease.checkpoint(new ExtendedSequenceNumber("1"))); final ArgumentCaptor leaseCreateCaptor = ArgumentCaptor.forClass(Lease.class); - final ArgumentCaptor leaseDeleteCaptor = ArgumentCaptor.forClass(Lease.class); when(shardDetector.listShards()).thenReturn(shards); when(dynamoDBLeaseRefresher.listLeases()).thenReturn(Collections.emptyList()) .thenReturn(Collections.emptyList()).thenReturn(leases); when(dynamoDBLeaseRefresher.createLeaseIfNotExists(leaseCreateCaptor.capture())) .thenThrow(new DependencyException(new Throwable("Throw for CreateLease"))).thenReturn(true); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseDeleteCaptor.capture()); try { // Initial call: No leases present, create leases. Create lease Fails hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(1)).listShards(); verify(dynamoDBLeaseRefresher, times(1)).listLeases(); @@ -1098,7 +976,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set createLeases = new HashSet<>(leaseCreateCaptor.getAllValues()); @@ -1114,27 +992,13 @@ public class HierarchicalShardSyncerTest { // Final call: Leases are present, shardId-0 is at ShardEnd needs to be cleaned up. hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, position, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - final List deleteLeases = leaseDeleteCaptor.getAllValues(); - final Set shardIds = deleteLeases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); - final Set sequenceNumbers = deleteLeases.stream().map(Lease::checkpoint) - .collect(Collectors.toSet()); - - final Set expectedShardIds = new HashSet<>( - Collections.singletonList(String.format(shardIdPrefix, 0))); - final Set expectedSequenceNumbers = new HashSet<>( - Collections.singletonList(ExtendedSequenceNumber.SHARD_END)); - - assertThat(deleteLeases.size(), equalTo(1)); - assertThat(shardIds, equalTo(expectedShardIds)); - assertThat(sequenceNumbers, equalTo(expectedSequenceNumbers)); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, times(3)).listShards(); verify(dynamoDBLeaseRefresher, times(1 + expectedCreateLeases.size())) .createLeaseIfNotExists(any(Lease.class)); verify(dynamoDBLeaseRefresher, times(3)).listLeases(); - verify(dynamoDBLeaseRefresher, times(1)).deleteLease(any(Lease.class)); + verify(dynamoDBLeaseRefresher, never()).deleteLease(any(Lease.class)); } } @@ -1186,64 +1050,6 @@ public class HierarchicalShardSyncerTest { }).collect(Collectors.toList()); } - @Test - public void testCleanUpGarbageLeaseForNonExistentShard() throws Exception { - final List shards = constructShardListForGraphA(); - final String garbageShardId = "shardId-garbage-001"; - final Shard garbageShard = ShardObjectHelper.newShard(garbageShardId, null, null, - ShardObjectHelper.newSequenceNumberRange("101", null)); - final Lease garbageLease = createLeaseFromShard(garbageShard, new ExtendedSequenceNumber("99"), LEASE_OWNER); - final List leases = new ArrayList<>( - createLeasesFromShards(shards, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER)); - leases.add(garbageLease); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeases()).thenReturn(leases); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); - - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); - assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); - - verify(shardDetector, times(2)).listShards(); - verify(dynamoDBLeaseRefresher).listLeases(); - verify(dynamoDBLeaseRefresher).deleteLease(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); - } - - @Test - public void testCleanUpGarbageLeaseForNonExistentShardForMultiStream() throws Exception { - final List shards = constructShardListForGraphA(); - final String garbageShardId = "shardId-garbage-001"; - final Shard garbageShard = ShardObjectHelper.newShard(garbageShardId, null, null, - ShardObjectHelper.newSequenceNumberRange("101", null)); - final Lease garbageLease = createMultiStreamLeaseFromShard(garbageShard, new ExtendedSequenceNumber("99"), LEASE_OWNER); - final List leases = new ArrayList<>( - createMultiStreamLeasesFromShards(shards, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER)); - leases.add(garbageLease); - - final ArgumentCaptor leaseCaptor = ArgumentCaptor.forClass(Lease.class); - - when(shardDetector.listShards()).thenReturn(shards); - when(dynamoDBLeaseRefresher.listLeasesForStream(any(StreamIdentifier.class))).thenReturn(leases); - doNothing().when(dynamoDBLeaseRefresher).deleteLease(leaseCaptor.capture()); - setupMultiStream(); - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, - INITIAL_POSITION_TRIM_HORIZON, SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); - - assertThat(leaseCaptor.getAllValues().size(), equalTo(1)); - assertThat(leaseCaptor.getValue(), equalTo(garbageLease)); - - verify(shardDetector, times(2)).listShards(); - verify(dynamoDBLeaseRefresher).listLeasesForStream(any(StreamIdentifier.class)); - verify(dynamoDBLeaseRefresher).deleteLease(any(Lease.class)); - verify(dynamoDBLeaseRefresher, never()).createLeaseIfNotExists(any(Lease.class)); - } - private void testCheckAndCreateLeasesForShardsIfMissing(InitialPositionInStreamExtended initialPosition) throws Exception { final String shardId0 = "shardId-0"; @@ -1279,7 +1085,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - SCOPE, cleanupLeasesOfCompletedShards, false, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); @@ -2473,7 +2279,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, initialPosition, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, atLeast(1)).listShardsWithFilter(shardFilter); verify(shardDetector, never()).listShards(); @@ -2495,7 +2301,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, atLeast(1)).listShards(); } @@ -2518,7 +2324,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector, times(3)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. @@ -2547,7 +2353,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, times(3)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. @@ -2570,7 +2376,7 @@ public class HierarchicalShardSyncerTest { hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, garbageCollectLeases, + SCOPE, ignoreUnexpectedChildShards, dynamoDBLeaseRefresher.isLeaseTableEmpty()); verify(shardDetector, times(1)).listShardsWithFilter(any(ShardFilter.class)); // Verify retries. From 6fc654802c5945f83dcda269334a5a9e53fe205d Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 11 Jun 2020 18:01:46 -0700 Subject: [PATCH 107/159] Limiting max page for ddb scan in isEmptyLeaseTable check to be 1 --- .../dynamodb/DynamoDBLeaseRefresher.java | 22 ++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) 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 30201236..df5746a2 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 @@ -292,7 +292,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(1, null).isEmpty(); + return list(1, 1, null).isEmpty(); } /** @@ -305,7 +305,23 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { * @throws DependencyException if DynamoDB scan fail in an unexpected way * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity */ - List list(Integer limit, StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, + List list(Integer limit, StreamIdentifier streamIdentifier) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + return list(limit, Integer.MAX_VALUE, streamIdentifier); + } + + /** + * List with the given page size. Package access for integration testing. + * + * @param limit number of items to consider at a time - used by integration tests to force paging. + * @param maxPages mad paginated scan calls + * @param streamIdentifier streamIdentifier for multi-stream mode. Can be null. + * @return list of leases + * @throws InvalidStateException if table does not exist + * @throws DependencyException if DynamoDB scan fail in an unexpected way + * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity + */ + private List list(Integer limit, Integer maxPages, StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException { log.debug("Listing leases from table {}", table); @@ -341,7 +357,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } Map lastEvaluatedKey = scanResult.lastEvaluatedKey(); - if (CollectionUtils.isNullOrEmpty(lastEvaluatedKey)) { + if (CollectionUtils.isNullOrEmpty(lastEvaluatedKey) || --maxPages <= 0) { // Signify that we're done. scanResult = null; log.debug("lastEvaluatedKey was null - scan finished."); From 2530481dbab357b87b74da50525c50a081c2ae99 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 12 Jun 2020 18:12:02 -0700 Subject: [PATCH 108/159] Periodic Shard Sync Auditor - Metrics and Customer auditor configs --- .../coordinator/PeriodicShardSyncManager.java | 61 +++++++++++++++---- .../amazon/kinesis/coordinator/Scheduler.java | 4 +- .../kinesis/leases/LeaseManagementConfig.java | 17 ++++++ .../PeriodicShardSyncManagerTest.java | 27 ++++---- 4 files changed, 82 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 c84547e2..fdc359bd 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 @@ -23,6 +23,7 @@ import lombok.Value; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.Validate; +import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.common.HashKeyRangeForLease; @@ -38,6 +39,11 @@ import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.lifecycle.TaskResult; +import software.amazon.kinesis.metrics.MetricsFactory; +import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; +import software.amazon.kinesis.metrics.MetricsUtil; +import software.amazon.kinesis.metrics.NullMetricsScope; import java.io.Serializable; import java.math.BigInteger; @@ -67,13 +73,11 @@ import static software.amazon.kinesis.common.HashKeyRangeForLease.fromHashKeyRan @Slf4j class PeriodicShardSyncManager { private static final long INITIAL_DELAY = 60 * 1000L; - private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L; @VisibleForTesting static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; @VisibleForTesting static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); - @VisibleForTesting - static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3; + static final String PERODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager"; private Map hashRangeHoleTrackerMap = new HashMap<>(); private final String workerId; @@ -83,19 +87,29 @@ class PeriodicShardSyncManager { private final Function shardSyncTaskManagerProvider; private final ScheduledExecutorService shardSyncThreadPool; private final boolean isMultiStreamingMode; + private final MetricsFactory metricsFactory; + private final long leasesRecoveryAuditorExecutionFrequencyMillis; + private final int leasesRecoveryAuditorInconsistencyConfidenceThreshold; private boolean isRunning; PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher, Map currentStreamConfigMap, - Function shardSyncTaskManagerProvider, boolean isMultiStreamingMode) { + Function shardSyncTaskManagerProvider, boolean isMultiStreamingMode, + MetricsFactory metricsFactory, + long leasesRecoveryAuditorExecutionFrequencyMillis, + int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { this(workerId, leaderDecider, leaseRefresher, currentStreamConfigMap, shardSyncTaskManagerProvider, - Executors.newSingleThreadScheduledExecutor(), isMultiStreamingMode); + Executors.newSingleThreadScheduledExecutor(), isMultiStreamingMode, metricsFactory, + leasesRecoveryAuditorExecutionFrequencyMillis, leasesRecoveryAuditorInconsistencyConfidenceThreshold); } PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, LeaseRefresher leaseRefresher, Map currentStreamConfigMap, Function shardSyncTaskManagerProvider, - ScheduledExecutorService shardSyncThreadPool, boolean isMultiStreamingMode) { + ScheduledExecutorService shardSyncThreadPool, boolean isMultiStreamingMode, + MetricsFactory metricsFactory, + long leasesRecoveryAuditorExecutionFrequencyMillis, + int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); this.workerId = workerId; @@ -105,6 +119,9 @@ class PeriodicShardSyncManager { this.shardSyncTaskManagerProvider = shardSyncTaskManagerProvider; this.shardSyncThreadPool = shardSyncThreadPool; this.isMultiStreamingMode = isMultiStreamingMode; + this.metricsFactory = metricsFactory; + this.leasesRecoveryAuditorExecutionFrequencyMillis = leasesRecoveryAuditorExecutionFrequencyMillis; + this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold; } public synchronized TaskResult start() { @@ -116,7 +133,7 @@ class PeriodicShardSyncManager { log.error("Error during runShardSync.", t); } }; - shardSyncThreadPool.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, + shardSyncThreadPool.scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, leasesRecoveryAuditorExecutionFrequencyMillis, TimeUnit.MILLISECONDS); isRunning = true; @@ -157,6 +174,13 @@ class PeriodicShardSyncManager { private void runShardSync() { if (leaderDecider.isLeader(workerId)) { log.info(String.format("WorkerId %s is leader, running the periodic shard sync task", workerId)); + + final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PERODIC_SHARD_SYNC_MANAGER); + int numStreamsWithPartialLeases = 0; + int numStreamsToSync = 0; + boolean isRunSuccess = false; + final long runStartMillis = System.currentTimeMillis(); + try { // Construct the stream to leases map to be used in the lease sync final Map> streamToLeasesMap = getStreamToLeasesMap( @@ -166,6 +190,10 @@ class PeriodicShardSyncManager { for (Map.Entry streamConfigEntry : currentStreamConfigMap.entrySet()) { final ShardSyncResponse shardSyncResponse = checkForShardSync(streamConfigEntry.getKey(), streamToLeasesMap.get(streamConfigEntry.getKey())); + + numStreamsWithPartialLeases += shardSyncResponse.isHoleDetected() ? 1 : 0; + numStreamsToSync += shardSyncResponse.shouldDoShardSync ? 1 : 0; + if (shardSyncResponse.shouldDoShardSync()) { log.info("Periodic shard syncer initiating shard sync for {} due to the reason - {} ", streamConfigEntry.getKey(), shardSyncResponse.reasonForDecision()); @@ -181,8 +209,14 @@ class PeriodicShardSyncManager { shardSyncResponse.reasonForDecision()); } } + isRunSuccess = true; } catch (Exception e) { log.error("Caught exception while running periodic shard syncer.", e); + } finally { + scope.addData("NumStreamsWithPartialLeases", numStreamsWithPartialLeases, StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("NumStreamsToSync", numStreamsToSync, StandardUnit.COUNT, MetricsLevel.SUMMARY); + MetricsUtil.addSuccessAndLatency(scope, isRunSuccess, runStartMillis, MetricsLevel.SUMMARY); + scope.end(); } } else { log.debug("WorkerId {} is not a leader, not running the shard sync task", workerId); @@ -214,7 +248,7 @@ class PeriodicShardSyncManager { if (CollectionUtils.isNullOrEmpty(leases)) { // If the leases is null or empty then we need to do shard sync log.info("No leases found for {}. Will be triggering shard sync", streamIdentifier); - return new ShardSyncResponse(true, "No leases found for " + streamIdentifier); + return new ShardSyncResponse(true, false,"No leases found for " + streamIdentifier); } // Check if there are any holes in the leases and return the first hole if present. Optional hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases); @@ -227,15 +261,15 @@ class PeriodicShardSyncManager { .computeIfAbsent(streamIdentifier, s -> new HashRangeHoleTracker()); final boolean hasHoleWithHighConfidence = hashRangeHoleTracker .hasHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); - return new ShardSyncResponse(hasHoleWithHighConfidence, + return new ShardSyncResponse(hasHoleWithHighConfidence, true, "Detected same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + " times. Shard sync will be initiated when threshold reaches " - + CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY); + + leasesRecoveryAuditorInconsistencyConfidenceThreshold); } else { // If hole is not present, clear any previous tracking for this stream and return false; hashRangeHoleTrackerMap.remove(streamIdentifier); - return new ShardSyncResponse(false, "Hash Ranges are complete for " + streamIdentifier); + return new ShardSyncResponse(false, false, "Hash Ranges are complete for " + streamIdentifier); } } @@ -244,6 +278,7 @@ class PeriodicShardSyncManager { @VisibleForTesting static class ShardSyncResponse { private final boolean shouldDoShardSync; + private final boolean isHoleDetected; private final String reasonForDecision; } @@ -365,7 +400,7 @@ class PeriodicShardSyncManager { private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; } - private static class HashRangeHoleTracker { + private class HashRangeHoleTracker { private HashRangeHole hashRangeHole; @Getter private Integer numConsecutiveHoles; @@ -377,7 +412,7 @@ class PeriodicShardSyncManager { this.hashRangeHole = hashRangeHole; this.numConsecutiveHoles = 1; } - return numConsecutiveHoles >= CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; + return numConsecutiveHoles >= leasesRecoveryAuditorInconsistencyConfidenceThreshold; } } 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 3e74e23b..e36f0582 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 @@ -290,7 +290,9 @@ public class Scheduler implements Runnable { this.schedulerInitializationBackoffTimeMillis = this.coordinatorConfig.schedulerInitializationBackoffTimeMillis(); this.leaderElectedPeriodicShardSyncManager = new PeriodicShardSyncManager( leaseManagementConfig.workerIdentifier(), leaderDecider, leaseRefresher, currentStreamConfigMap, - shardSyncTaskManagerProvider, isMultiStreamMode); + shardSyncTaskManagerProvider, isMultiStreamMode, metricsFactory, + leaseManagementConfig.leasesRecoveryAuditorExecutionFrequencyMillis(), + leaseManagementConfig.leasesRecoveryAuditorInconsistencyConfidenceThreshold()); this.leaseCleanupManager = this.leaseManagementConfig.leaseManagementFactory(leaseSerializer, isMultiStreamMode) .createLeaseCleanupManager(metricsFactory); } 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 82c02060..473db5bb 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseManagementConfig.java @@ -52,6 +52,9 @@ public class LeaseManagementConfig { public static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(1).toMillis(); public static final long DEFAULT_COMPLETED_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(5).toMillis(); public static final long DEFAULT_GARBAGE_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(30).toMillis(); + public static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L; + public static final int DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY = 3; + public static final LeaseCleanupConfig DEFAULT_LEASE_CLEANUP_CONFIG = LeaseCleanupConfig.builder() .leaseCleanupIntervalMillis(DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS) @@ -195,6 +198,20 @@ public class LeaseManagementConfig { private BillingMode billingMode = BillingMode.PROVISIONED; + /** + * Frequency (in millis) of the auditor job to scan for partial leases in the lease table. + * If the auditor detects any hole in the leases for a stream, then it would trigger shard sync based on + * {@link #leasesRecoveryAuditorInconsistencyConfidenceThreshold} + */ + private long leasesRecoveryAuditorExecutionFrequencyMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS; + + /** + * Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table + * is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, + * then it would trigger a shard sync. + */ + private int leasesRecoveryAuditorInconsistencyConfidenceThreshold = DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY; + /** * The initial position for getting records from Kinesis streams. * 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 dfba2791..a2047a6b 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 @@ -34,6 +34,7 @@ import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.MultiStreamLease; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardSyncTaskManager; +import software.amazon.kinesis.metrics.NullMetricsFactory; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.math.BigInteger; @@ -49,9 +50,9 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static software.amazon.kinesis.common.HashKeyRangeForLease.deserialize; -import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MAX_HASH_KEY; import static software.amazon.kinesis.coordinator.PeriodicShardSyncManager.MIN_HASH_KEY; +import static software.amazon.kinesis.leases.LeaseManagementConfig.DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY; @RunWith(MockitoJUnitRunner.class) @@ -72,7 +73,7 @@ public class PeriodicShardSyncManagerTest { public void setup() { streamIdentifier = StreamIdentifier.multiStreamInstance("123:stream:456"); periodicShardSyncManager = new PeriodicShardSyncManager("worker", leaderDecider, leaseRefresher, currentStreamConfigMap, - shardSyncTaskManagerProvider, true); + shardSyncTaskManagerProvider, true, new NullMetricsFactory(), 2 * 60 * 1000, 3); } @Test @@ -173,7 +174,7 @@ public class PeriodicShardSyncManagerTest { lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); return lease; }).collect(Collectors.toList()); - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); } @@ -191,7 +192,7 @@ public class PeriodicShardSyncManagerTest { lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); return lease; }).collect(Collectors.toList()); - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @@ -214,7 +215,7 @@ public class PeriodicShardSyncManagerTest { } return lease; }).collect(Collectors.toList()); - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @@ -233,7 +234,7 @@ public class PeriodicShardSyncManagerTest { lease.checkpoint(ExtendedSequenceNumber.SHARD_END); return lease; }).collect(Collectors.toList()); - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @@ -252,7 +253,7 @@ public class PeriodicShardSyncManagerTest { lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); return lease; }).collect(Collectors.toList()); - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); List multiStreamLeases2 = new ArrayList() {{ add(deserialize(MIN_HASH_KEY.toString(), "1")); @@ -267,7 +268,7 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); // Resetting the holes - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync())); Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync()); } @@ -286,7 +287,7 @@ public class PeriodicShardSyncManagerTest { lease.checkpoint(ExtendedSequenceNumber.TRIM_HORIZON); return lease; }).collect(Collectors.toList()); - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); List multiStreamLeases2 = new ArrayList() {{ add(deserialize(MIN_HASH_KEY.toString(), "1")); @@ -301,10 +302,10 @@ public class PeriodicShardSyncManagerTest { return lease; }).collect(Collectors.toList()); // Resetting the holes - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases2).shouldDoShardSync())); // Resetting the holes - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); } @@ -347,7 +348,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); // Assert that shard sync should never trigger - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); Assert.assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); @@ -395,7 +396,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); // Assert that shard sync should never trigger - IntStream.range(1, CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY).forEach(i -> Assert + IntStream.range(1, DEFAULT_CONSECUTIVE_HOLES_FOR_TRIGGERING_LEASE_RECOVERY).forEach(i -> Assert .assertFalse(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync())); Assert.assertTrue(periodicShardSyncManager.checkForShardSync(streamIdentifier, multiStreamLeases).shouldDoShardSync()); From ce381783996ff1307b96b31bbaad0b1606a8b33e Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 12 Jun 2020 22:20:38 -0700 Subject: [PATCH 109/159] Caching consumerArn for StreamIdentifier in FanOutRetrievalFactory --- .../fanout/FanOutRetrievalFactory.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) 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 5796862b..35301624 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRetrievalFactory.java @@ -38,9 +38,11 @@ public class FanOutRetrievalFactory implements RetrievalFactory { private final KinesisAsyncClient kinesisClient; private final String defaultStreamName; - private final String defaultConsumerName; + private final String defaultConsumerArn; private final Function consumerArnCreator; + private Map implicitConsumerArnTracker = new HashMap<>(); + @Override public GetRecordsRetrievalStrategy createGetRecordsRetrievalStrategy(final ShardInfo shardInfo, final MetricsFactory metricsFactory) { @@ -52,15 +54,15 @@ public class FanOutRetrievalFactory implements RetrievalFactory { final StreamConfig streamConfig, final MetricsFactory metricsFactory) { final Optional streamIdentifierStr = shardInfo.streamIdentifierSerOpt(); - final String streamName; if(streamIdentifierStr.isPresent()) { - streamName = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()).streamName(); + final StreamIdentifier streamIdentifier = StreamIdentifier.multiStreamInstance(streamIdentifierStr.get()); return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), - getOrCreateConsumerArn(streamName, streamConfig.consumerArn()), + getOrCreateConsumerArn(streamIdentifier, streamConfig.consumerArn()), streamIdentifierStr.get()); } else { + final StreamIdentifier streamIdentifier = StreamIdentifier.singleStreamInstance(defaultStreamName); return new FanOutRecordsPublisher(kinesisClient, shardInfo.shardId(), - getOrCreateConsumerArn(defaultStreamName, defaultConsumerName)); + getOrCreateConsumerArn(streamIdentifier, defaultConsumerArn)); } } @@ -69,7 +71,8 @@ public class FanOutRetrievalFactory implements RetrievalFactory { throw new UnsupportedOperationException("FanoutRetrievalFactory needs StreamConfig Info"); } - private String getOrCreateConsumerArn(String streamName, String consumerArn) { - return consumerArn != null ? consumerArn : consumerArnCreator.apply(streamName); + private String getOrCreateConsumerArn(StreamIdentifier streamIdentifier, String consumerArn) { + return consumerArn != null ? consumerArn : implicitConsumerArnTracker + .computeIfAbsent(streamIdentifier, sId -> consumerArnCreator.apply(sId.streamName())); } } From c1cbb6cf6d9761444d51a128b8412df93a3c3030 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Fri, 12 Jun 2020 09:28:30 -0700 Subject: [PATCH 110/159] Removing one of the lease clean up strategy as it is no longer required with distributed lease cleanup --- .../amazon/kinesis/coordinator/Scheduler.java | 12 ++-- .../FormerStreamsLeasesDeletionStrategy.java | 55 ++----------------- .../kinesis/coordinator/SchedulerTest.java | 12 ++-- 3 files changed, 16 insertions(+), 63 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 e36f0582..b6efc793 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 @@ -113,6 +113,7 @@ public class Scheduler implements Runnable { private static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; private static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; private static final long NEW_STREAM_CHECK_INTERVAL_MILLIS = 1 * 60 * 1000L; + private static final boolean SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS = false; private static final String MULTI_STREAM_TRACKER = "MultiStreamTracker"; private static final String ACTIVE_STREAMS_COUNT = "ActiveStreams.Count"; private static final String PENDING_STREAMS_DELETION_COUNT = "StreamsPendingDeletion.Count"; @@ -491,10 +492,9 @@ public class Scheduler implements Runnable { } }; - if (formerStreamsLeasesDeletionStrategy.shouldCleanupLeasesForDeletedStreams()) { + if (SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS) { // We do lease sync for old streams, before leaving to the deletion strategy to delete leases for - // strategy detected leases. Also, for deleted streams we expect the shard sync to remove the - // leases. + // strategy detected leases. Iterator currentSetOfStreamsIter = currentStreamConfigMap.keySet().iterator(); while (currentSetOfStreamsIter.hasNext()) { StreamIdentifier streamIdentifier = currentSetOfStreamsIter.next(); @@ -533,13 +533,13 @@ public class Scheduler implements Runnable { currentStreamConfigMap.keySet().stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier)); } else if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION) { - Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiers()).ifPresent( + Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiersForLeaseCleanup()).ifPresent( streamIdentifiers -> streamIdentifiers.stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier))); } - // Now let's scan the streamIdentifiers eligible for deferred deletion and delete them. + // 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 streamIdentifiers are not present in the latest snapshot. + // the streamIdentifiersForLeaseCleanup are not present in the latest snapshot. final Map> staleStreamIdDeletionDecisionMap = staleStreamDeletionMap.keySet().stream().collect(Collectors .partitioningBy(streamIdentifier -> newStreamConfigMap.containsKey(streamIdentifier), Collectors.toSet())); final Set staleStreamIdsToBeDeleted = staleStreamIdDeletionDecisionMap.get(false).stream().filter(streamIdentifier -> diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java index e59266a4..232c428d 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/processor/FormerStreamsLeasesDeletionStrategy.java @@ -29,7 +29,7 @@ public interface FormerStreamsLeasesDeletionStrategy { * StreamIdentifiers for which leases needs to be cleaned up in the lease table. * @return */ - List streamIdentifiers(); + List streamIdentifiersForLeaseCleanup(); /** * Duration to wait before deleting the leases for this stream. @@ -43,12 +43,6 @@ public interface FormerStreamsLeasesDeletionStrategy { */ StreamsLeasesDeletionType leaseDeletionType(); - /** - * Should the leases be cleaned up for deleted streams - * @return true if leases be cleaned up for deleted streams; false otherwise. - */ - boolean shouldCleanupLeasesForDeletedStreams(); - /** * StreamsLeasesDeletionType identifying the different lease cleanup strategies. */ @@ -64,7 +58,7 @@ public interface FormerStreamsLeasesDeletionStrategy { final class NoLeaseDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { @Override - public final List streamIdentifiers() { + public final List streamIdentifiersForLeaseCleanup() { throw new UnsupportedOperationException("StreamIdentifiers not required"); } @@ -77,37 +71,6 @@ public interface FormerStreamsLeasesDeletionStrategy { public final StreamsLeasesDeletionType leaseDeletionType() { return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION; } - - @Override - public final boolean shouldCleanupLeasesForDeletedStreams() { - return false; - } - } - - /** - * Strategy for not cleaning up leases for former streams. - */ - final class OnlyDeletedStreamsLeasesCleanupStrategy implements FormerStreamsLeasesDeletionStrategy { - - @Override - public final List streamIdentifiers() { - throw new UnsupportedOperationException("StreamIdentifiers not required"); - } - - @Override - public final Duration waitPeriodToDeleteFormerStreams() { - return Duration.ZERO; - } - - @Override - public final StreamsLeasesDeletionType leaseDeletionType() { - return StreamsLeasesDeletionType.NO_STREAMS_LEASES_DELETION; - } - - @Override - public final boolean shouldCleanupLeasesForDeletedStreams() { - return true; - } } /** @@ -117,7 +80,7 @@ public interface FormerStreamsLeasesDeletionStrategy { abstract class AutoDetectionAndDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { @Override - public final List streamIdentifiers() { + public final List streamIdentifiersForLeaseCleanup() { throw new UnsupportedOperationException("StreamIdentifiers not required"); } @@ -125,15 +88,10 @@ public interface FormerStreamsLeasesDeletionStrategy { public final StreamsLeasesDeletionType leaseDeletionType() { return StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION; } - - @Override - public boolean shouldCleanupLeasesForDeletedStreams() { - return false; - } } /** - * Strategy to detect the streams for deletion through {@link #streamIdentifiers()} provided by customer at runtime + * Strategy to detect the streams for deletion through {@link #streamIdentifiersForLeaseCleanup()} provided by customer at runtime * and do deferred deletion based on {@link #waitPeriodToDeleteFormerStreams()} */ abstract class ProvidedStreamsDeferredDeletionStrategy implements FormerStreamsLeasesDeletionStrategy { @@ -142,11 +100,6 @@ public interface FormerStreamsLeasesDeletionStrategy { public final StreamsLeasesDeletionType leaseDeletionType() { return StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION; } - - @Override - public boolean shouldCleanupLeasesForDeletedStreams() { - return false; - } } } 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 af58d3ab..9e15d988 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 @@ -482,7 +482,7 @@ public class SchedulerTest { public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { - @Override public List streamIdentifiers() { + @Override public List streamIdentifiersForLeaseCleanup() { return null; } @@ -497,7 +497,7 @@ public class SchedulerTest { public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyProvidedListStrategy2() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { - @Override public List streamIdentifiers() { + @Override public List streamIdentifiersForLeaseCleanup() { return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( Collectors.toCollection(ArrayList::new)); @@ -555,7 +555,7 @@ public class SchedulerTest { public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { - @Override public List streamIdentifiers() { + @Override public List streamIdentifiersForLeaseCleanup() { return null; } @@ -575,7 +575,7 @@ public class SchedulerTest { public final void testMultiStreamStaleStreamsAreDeletedAfterDefermentPeriodWithProvidedListStrategy2() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { - @Override public List streamIdentifiers() { + @Override public List streamIdentifiersForLeaseCleanup() { return IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( Collectors.toCollection(ArrayList::new)); @@ -639,7 +639,7 @@ public class SchedulerTest { public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { - @Override public List streamIdentifiers() { + @Override public List streamIdentifiersForLeaseCleanup() { return null; } @@ -654,7 +654,7 @@ public class SchedulerTest { public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithProvidedListStrategy2() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new ProvidedStreamsDeferredDeletionStrategy() { - @Override public List streamIdentifiers() { + @Override public List streamIdentifiersForLeaseCleanup() { return IntStream.range(1, 3) .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) From aaf37be0cdd77733c637b0e8f7ba1c42aa3e7433 Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Mon, 15 Jun 2020 13:59:43 -0400 Subject: [PATCH 111/159] Convert expiredLeases to set (#56) --- .../amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java index d79646e7..41147947 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 @@ -499,10 +499,13 @@ public class DynamoDBLeaseTaker implements LeaseTaker { */ private Map computeLeaseCounts(List expiredLeases) { Map leaseCounts = new HashMap<>(); + // The set will give much faster lookup than the original list, an + // important optimization when the list is large + Set expiredLeasesSet = new HashSet<>(expiredLeases); // Compute the number of leases per worker by looking through allLeases and ignoring leases that have expired. for (Lease lease : allLeases.values()) { - if (!expiredLeases.contains(lease)) { + if (!expiredLeasesSet.contains(lease)) { String leaseOwner = lease.leaseOwner(); Integer oldCount = leaseCounts.get(leaseOwner); if (oldCount == null) { From 37281e949399563e2dccae4b83c7f90f32fcad42 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 15 Jun 2020 20:57:55 -0700 Subject: [PATCH 112/159] Deleting lease immediately in RNF Exception --- .../kinesis/leases/LeaseCleanupManager.java | 43 +++++++++++---- .../kinesis/lifecycle/ShutdownTask.java | 55 ++++++++++++++----- 2 files changed, 74 insertions(+), 24 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 6d6b46a8..4fda2106 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 @@ -44,6 +44,7 @@ import software.amazon.kinesis.retrieval.AWSExceptionManager; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.time.Duration; +import java.util.Collections; import java.util.HashSet; import java.util.Objects; import java.util.Optional; @@ -116,13 +117,24 @@ public class LeaseCleanupManager { //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597. if (!deletionQueue.contains(leasePendingDeletion)) { log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey()); - deletionQueue.add(leasePendingDeletion); + if (!deletionQueue.add(leasePendingDeletion)) { + log.warn("Unable to enqueue lease {} for deletion.", lease.leaseKey()); + } } else { log.warn("Lease {} is already pending deletion, not enqueueing for deletion.", lease.leaseKey()); } } } + /** + * Check if lease was already enqueued for deletion. + * @param leasePendingDeletion + * @return true if enqueued for deletion; false otherwise. + */ + public boolean isEnqueuedForDeletion(LeasePendingDeletion leasePendingDeletion) { + return deletionQueue.contains(leasePendingDeletion); + } + /** * Returns how many leases are currently waiting in the queue pending deletion. * @return number of leases pending deletion. @@ -139,7 +151,8 @@ public class LeaseCleanupManager { return garbageLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= garbageLeaseCleanupIntervalMillis; } - private LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion) throws TimeoutException, + public LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion, + boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException, InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { final Lease lease = leasePendingDeletion.lease(); final ShardInfo shardInfo = leasePendingDeletion.shardInfo(); @@ -150,9 +163,11 @@ public class LeaseCleanupManager { boolean cleanedUpCompletedLease = false; boolean cleanedUpGarbageLease = false; boolean alreadyCheckedForGarbageCollection = false; + boolean wereChildShardsPresent = false; + boolean wasResourceNotFound = false; try { - if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard()) { + if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) { Set childShardKeys = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey()).childShardIds(); if (CollectionUtils.isNullOrEmpty(childShardKeys)) { try { @@ -172,18 +187,21 @@ public class LeaseCleanupManager { cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys); } - if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard()) { + if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) { try { - getChildShardsFromService(shardInfo, streamIdentifier); + wereChildShardsPresent = !CollectionUtils + .isNullOrEmpty(getChildShardsFromService(shardInfo, streamIdentifier)); } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } } } catch (ResourceNotFoundException e) { + wasResourceNotFound = true; cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease); } - return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease); + return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease, wereChildShardsPresent, + wasResourceNotFound); } private Set getChildShardsFromService(ShardInfo shardInfo, StreamIdentifier streamIdentifier) @@ -289,22 +307,23 @@ public class LeaseCleanupManager { final LeasePendingDeletion leasePendingDeletion = deletionQueue.poll(); final String leaseKey = leasePendingDeletion.lease().leaseKey(); final StreamIdentifier streamIdentifier = leasePendingDeletion.streamIdentifier(); - boolean deletionFailed = true; + boolean deletionSucceeded = false; try { - final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion); + final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion, + timeToCheckForCompletedShard(), timeToCheckForGarbageShard()); completedLeaseCleanedUp |= leaseCleanupResult.cleanedUpCompletedLease(); garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease(); if (leaseCleanupResult.leaseCleanedUp()) { log.debug("Successfully cleaned up lease {} for {}", leaseKey, streamIdentifier); - deletionFailed = false; + deletionSucceeded = true; } } catch (Exception e) { log.error("Failed to cleanup lease {} for {}. Will re-enqueue for deletion and retry on next " + "scheduled execution.", leaseKey, streamIdentifier, e); } - if (deletionFailed) { + if (!deletionSucceeded) { log.debug("Did not cleanup lease {} for {}. Re-enqueueing for deletion.", leaseKey, streamIdentifier); failedDeletions.add(leasePendingDeletion); } @@ -332,9 +351,11 @@ public class LeaseCleanupManager { } @Value - private class LeaseCleanupResult { + public static class LeaseCleanupResult { boolean cleanedUpCompletedLease; boolean cleanedUpGarbageLease; + boolean wereChildShardsPresent; + boolean wasResourceNotFound; public boolean leaseCleanedUp() { return cleanedUpCompletedLease | cleanedUpGarbageLease; diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 5f1dcd25..73745168 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 @@ -128,21 +128,50 @@ public class ShutdownTask implements ConsumerTask { if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); updateLeaseWithChildShards(currentShardLease); - } - final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) - .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); - if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { - recordProcessorCheckpointer - .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); - recordProcessorCheckpointer.largestPermittedCheckpointValue(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); - } + final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) + .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); + if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { + recordProcessorCheckpointer + .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); + recordProcessorCheckpointer.largestPermittedCheckpointValue(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); + } - final LeasePendingDeletion garbageLease = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); - leaseCleanupManager.enqueueForDeletion(garbageLease); + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + + } else { + // This might be a case of ResourceNotFound from Service. Directly validate and delete lease, if required. + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, + currentShardLease, shardInfo); + if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { + final LeaseCleanupManager.LeaseCleanupResult leaseCleanupResult; + try { + leaseCleanupResult = leaseCleanupManager + .cleanupLease(leasePendingDeletion, false, true); + if (leaseCleanupResult.leaseCleanedUp()) { + log.info("Cleaned up garbage lease {} for {}. Details : {}", + currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); + } else { + log.error("Unable to cleanup garbage lease {} for {}. Details : {} ", + currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); + // If we are unable to delete this lease and the reason being RNF, then enqueue it + // for deletion, so that we don't end up consuming service TPS on any bugs. + if (leaseCleanupResult.wasResourceNotFound()) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + } + } + } catch (Exception e) { + log.error("Unable to cleanup garbage lease {} for {}", currentShardLease.leaseKey(), + streamIdentifier, e); + } finally { + + } + } + } } else { throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); } From b60dd60f35b956b7d97af499050a68e28100a593 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 15 Jun 2020 21:20:33 -0700 Subject: [PATCH 113/159] Code refactoring - 1 --- .../leases/dynamodb/DynamoDBLeaseRenewer.java | 14 ++++++-------- .../amazon/kinesis/lifecycle/ShutdownTask.java | 8 +++----- 2 files changed, 9 insertions(+), 13 deletions(-) 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 ecb0fc26..e457b5ec 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRenewer.java @@ -298,14 +298,12 @@ public class DynamoDBLeaseRenewer implements LeaseRenewer { } final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, operation); - if (StringUtils.isNotEmpty(singleStreamShardId)) { - if(lease instanceof MultiStreamLease) { - MetricsUtil.addStreamId(scope, - StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())); - MetricsUtil.addShardId(scope, ((MultiStreamLease) lease).shardId()); - } else { - MetricsUtil.addShardId(scope, singleStreamShardId); - } + if (lease instanceof MultiStreamLease) { + MetricsUtil.addStreamId(scope, + StreamIdentifier.multiStreamInstance(((MultiStreamLease) lease).streamIdentifier())); + MetricsUtil.addShardId(scope, ((MultiStreamLease) lease).shardId()); + } else if (StringUtils.isNotEmpty(singleStreamShardId)) { + MetricsUtil.addShardId(scope, singleStreamShardId); } long startTime = System.currentTimeMillis(); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 73745168..64e394b3 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 @@ -132,9 +132,6 @@ public class ShutdownTask implements ConsumerTask { final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { - recordProcessorCheckpointer - .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); - recordProcessorCheckpointer.largestPermittedCheckpointValue(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); @@ -167,8 +164,6 @@ public class ShutdownTask implements ConsumerTask { } catch (Exception e) { log.error("Unable to cleanup garbage lease {} for {}", currentShardLease.leaseKey(), streamIdentifier, e); - } finally { - } } } @@ -203,6 +198,9 @@ public class ShutdownTask implements ConsumerTask { } private void applicationCheckpointAndVerification() { + recordProcessorCheckpointer + .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); + recordProcessorCheckpointer.largestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); shardRecordProcessor.shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.lastCheckpointValue(); if (lastCheckpointValue == null From 8ec14baca1a31d9334b4782adf43578bba0f8936 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 15 Jun 2020 21:53:30 -0700 Subject: [PATCH 114/159] Handlign garbage shard case while delivering lease cleanup --- .../kinesis/lifecycle/ShutdownTask.java | 78 ++++++++++++------- 1 file changed, 48 insertions(+), 30 deletions(-) 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 64e394b3..9c77d910 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 @@ -124,46 +124,29 @@ 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. final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, + currentShardLease, shardInfo); if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); updateLeaseWithChildShards(currentShardLease); - - final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) - .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); - if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { - // Call the shardRecordsProcessor to checkpoint with SHARD_END sequence number. - // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded. - throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime); - } - - final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); + // Attempt to do shard checkpointing and throw on exception. + attemptShardEndCheckpointing(scope, startTime); + // Enqueue completed shard for deletion. leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); } else { // This might be a case of ResourceNotFound from Service. Directly validate and delete lease, if required. - final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, - currentShardLease, shardInfo); + // If already enqueued for deletion as part of this worker, do not attempt to shard end checkpoint + // or lease cleanup. Else try to shard end checkpoint and cleanup the lease if the shard is a + // garbage shard. if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { - final LeaseCleanupManager.LeaseCleanupResult leaseCleanupResult; try { - leaseCleanupResult = leaseCleanupManager - .cleanupLease(leasePendingDeletion, false, true); - if (leaseCleanupResult.leaseCleanedUp()) { - log.info("Cleaned up garbage lease {} for {}. Details : {}", - currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); - } else { - log.error("Unable to cleanup garbage lease {} for {}. Details : {} ", - currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); - // If we are unable to delete this lease and the reason being RNF, then enqueue it - // for deletion, so that we don't end up consuming service TPS on any bugs. - if (leaseCleanupResult.wasResourceNotFound()) { - leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); - } - } - } catch (Exception e) { - log.error("Unable to cleanup garbage lease {} for {}", currentShardLease.leaseKey(), - streamIdentifier, e); + // Do a best effort shard end checkpointing, before attempting to cleanup the lease, + // in the case of RNF Exception. + attemptShardEndCheckpointing(scope, startTime); + } finally { + attemptGarbageCollectionOfLeaseAndEnqueueOnFailure(leasePendingDeletion, currentShardLease); } } } @@ -197,6 +180,41 @@ public class ShutdownTask implements ConsumerTask { return new TaskResult(exception); } + private void attemptShardEndCheckpointing(MetricsScope scope, long startTime) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, + CustomerApplicationException { + final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) + .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); + if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { + // Call the shardRecordsProcessor to checkpoint with SHARD_END sequence number. + // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded. + throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime); + } + } + + private void attemptGarbageCollectionOfLeaseAndEnqueueOnFailure(LeasePendingDeletion leasePendingDeletion, Lease currentShardLease) { + final LeaseCleanupManager.LeaseCleanupResult leaseCleanupResult; + try { + leaseCleanupResult = leaseCleanupManager + .cleanupLease(leasePendingDeletion, false, true); + if (leaseCleanupResult.leaseCleanedUp()) { + log.info("Cleaned up garbage lease {} for {}. Details : {}", + currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); + } else { + log.error("Unable to cleanup potential garbage lease {} for {}. Details : {} ", + currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); + // If we are unable to delete this lease and the reason being RNF, then enqueue it + // for deletion, so that we don't end up consuming service TPS on any bugs. + if (leaseCleanupResult.wasResourceNotFound()) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + } + } + } catch (Exception e) { + log.error("Unable to cleanup potential garbage lease {} for {}", currentShardLease.leaseKey(), + streamIdentifier, e); + } + } + private void applicationCheckpointAndVerification() { recordProcessorCheckpointer .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); From 7b0dc0d3e58be27c49f29ebab6caf72f73c64af1 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 15 Jun 2020 22:00:56 -0700 Subject: [PATCH 115/159] Added comments on the garbage cleanup logic --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 2 ++ 1 file changed, 2 insertions(+) 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 9c77d910..dd98ca87 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 @@ -146,6 +146,8 @@ public class ShutdownTask implements ConsumerTask { // in the case of RNF Exception. attemptShardEndCheckpointing(scope, startTime); } finally { + // If we don't want to cleanup the garbage shard without successful shard end + // checkpointing, remove the try finally construct and only execute the methods. attemptGarbageCollectionOfLeaseAndEnqueueOnFailure(leasePendingDeletion, currentShardLease); } } From 4d65f5603899d8389aadd5b74d4783ed932e577e Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 15 Jun 2020 22:02:33 -0700 Subject: [PATCH 116/159] Added comments on the garbage cleanup logic - 2 --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 1 + 1 file changed, 1 insertion(+) 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 dd98ca87..9743704d 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 @@ -146,6 +146,7 @@ public class ShutdownTask implements ConsumerTask { // in the case of RNF Exception. attemptShardEndCheckpointing(scope, startTime); } finally { + // Attempt to garbage collect if this shard is no longer associated with the stream. // If we don't want to cleanup the garbage shard without successful shard end // checkpointing, remove the try finally construct and only execute the methods. attemptGarbageCollectionOfLeaseAndEnqueueOnFailure(leasePendingDeletion, currentShardLease); From 374e47b208afccfbb0ca346bb73e54b12d270437 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 15 Jun 2020 23:13:47 -0700 Subject: [PATCH 117/159] updating wereChildShardsPresent hint in all cases --- .../software/amazon/kinesis/leases/LeaseCleanupManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 4fda2106..0a2b65eb 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 @@ -173,9 +173,10 @@ public class LeaseCleanupManager { try { childShardKeys = getChildShardsFromService(shardInfo, streamIdentifier); - if (childShardKeys == null) { + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { log.error("No child shards returned from service for shard {} for {}.", shardInfo.shardId(), streamIdentifier.streamName()); } else { + wereChildShardsPresent = true; updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); } } catch (ExecutionException e) { @@ -183,6 +184,8 @@ public class LeaseCleanupManager { } finally { alreadyCheckedForGarbageCollection = true; } + } else { + wereChildShardsPresent = true; } cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys); } From e7e8196f29283cee0daca251a1afb11a0777e9da Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 16 Jun 2020 10:09:13 -0700 Subject: [PATCH 118/159] Fixed typo --- .../kinesis/coordinator/PeriodicShardSyncManager.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 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 fdc359bd..b4999bec 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 @@ -43,7 +43,6 @@ import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.metrics.MetricsLevel; import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; -import software.amazon.kinesis.metrics.NullMetricsScope; import java.io.Serializable; import java.math.BigInteger; @@ -53,7 +52,6 @@ import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.Executors; @@ -77,7 +75,7 @@ class PeriodicShardSyncManager { static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; @VisibleForTesting static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); - static final String PERODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager"; + static final String PERIODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager"; private Map hashRangeHoleTrackerMap = new HashMap<>(); private final String workerId; @@ -175,7 +173,8 @@ class PeriodicShardSyncManager { if (leaderDecider.isLeader(workerId)) { log.info(String.format("WorkerId %s is leader, running the periodic shard sync task", workerId)); - final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, PERODIC_SHARD_SYNC_MANAGER); + final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, + PERIODIC_SHARD_SYNC_MANAGER); int numStreamsWithPartialLeases = 0; int numStreamsToSync = 0; boolean isRunSuccess = false; @@ -248,7 +247,7 @@ class PeriodicShardSyncManager { if (CollectionUtils.isNullOrEmpty(leases)) { // If the leases is null or empty then we need to do shard sync log.info("No leases found for {}. Will be triggering shard sync", streamIdentifier); - return new ShardSyncResponse(true, false,"No leases found for " + streamIdentifier); + return new ShardSyncResponse(true, false, "No leases found for " + streamIdentifier); } // Check if there are any holes in the leases and return the first hole if present. Optional hashRangeHoleOpt = hasHoleInLeases(streamIdentifier, leases); From 4a89c26f6f73f1051b82149633684c53878fe112 Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Tue, 5 May 2020 21:06:10 -0700 Subject: [PATCH 119/159] Empty lease response for ResourceNotFound exceptions (#27) --- .../kinesis/leases/KinesisShardDetector.java | 10 ++++- .../leases/KinesisShardDetectorTest.java | 39 ++++++++----------- 2 files changed, 26 insertions(+), 23 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 9eb23c85..f5ef482e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -19,6 +19,7 @@ import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -39,6 +40,7 @@ import software.amazon.awssdk.services.kinesis.model.LimitExceededException; import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; import software.amazon.awssdk.utils.CollectionUtils; @@ -179,6 +181,7 @@ 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); @@ -194,7 +197,6 @@ public class KinesisShardDetector implements ShardDetector { int remainingRetries = maxListShardsRetryAttempts; while (result == null) { - try { try { result = getListShardsResponse(request.build()); @@ -218,6 +220,12 @@ public class KinesisShardDetector implements ShardDetector { log.debug("Stream {} : Sleep was interrupted ", streamIdentifier, ie); } lastException = e; + } catch (ResourceNotFoundException e) { + log.warn("Got ResourceNotFoundException when fetching shard list for {}. Stream no longer exists.", + streamIdentifier.streamName()); + return ListShardsResponse.builder().shards(Collections.emptyList()) + .nextToken(null) + .build(); } catch (TimeoutException te) { throw new RuntimeException(te); } diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java index 1a37f614..68bb7d97 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java @@ -15,18 +15,6 @@ package software.amazon.kinesis.leases; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.isA; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.junit.Assert.assertThat; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -35,7 +23,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.IntStream; - +import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -43,7 +31,6 @@ import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; - import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.services.kinesis.model.LimitExceededException; import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; @@ -51,6 +38,16 @@ import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.Shard; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.isA; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * @@ -143,19 +140,17 @@ public class KinesisShardDetectorTest { } } - @Test(expected = ResourceNotFoundException.class) - public void testListShardsResourceNotFound() { + @Test + public void testListShardsResourceNotFoundReturnsEmptyResponse() { final CompletableFuture future = CompletableFuture.supplyAsync(() -> { throw ResourceNotFoundException.builder().build(); }); - when(client.listShards(any(ListShardsRequest.class))).thenReturn(future); - try { - shardDetector.listShards(); - } finally { - verify(client).listShards(any(ListShardsRequest.class)); - } + List shards = shardDetector.listShards(); + + Assert.assertEquals(0, shards.size()); + verify(client).listShards(any(ListShardsRequest.class)); } @Test From 69a05b409b6a28672b1640252c83b2eea915d39b Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 17 Jun 2020 10:25:11 -0700 Subject: [PATCH 120/159] Removing the old streams from active streams list without cleaning up the leases --- .../amazon/kinesis/coordinator/Scheduler.java | 41 ++++++------ .../kinesis/coordinator/SchedulerTest.java | 64 +++++++++++++------ 2 files changed, 66 insertions(+), 39 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 b6efc793..6a87db23 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 @@ -492,24 +492,6 @@ public class Scheduler implements Runnable { } }; - if (SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS) { - // We do lease sync for old streams, before leaving to the deletion strategy to delete leases for - // strategy detected leases. - Iterator currentSetOfStreamsIter = currentStreamConfigMap.keySet().iterator(); - while (currentSetOfStreamsIter.hasNext()) { - StreamIdentifier streamIdentifier = currentSetOfStreamsIter.next(); - if (!newStreamConfigMap.containsKey(streamIdentifier)) { - log.info("Found old/deleted stream: " + streamIdentifier - + ". Syncing shards of that stream."); - ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager( - currentStreamConfigMap.get(streamIdentifier)); - shardSyncTaskManager.submitShardSyncTask(); - currentSetOfStreamsIter.remove(); - streamsSynced.add(streamIdentifier); - } - } - } - if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.FORMER_STREAMS_AUTO_DETECTION_DEFERRED_DELETION) { // Now, we are identifying the stale/old streams and enqueuing it for deferred deletion. // It is assumed that all the workers will always have the latest and consistent snapshot of streams @@ -535,6 +517,29 @@ public class Scheduler implements Runnable { } else if (formerStreamsLeasesDeletionStrategy.leaseDeletionType() == StreamsLeasesDeletionType.PROVIDED_STREAMS_DEFERRED_DELETION) { Optional.ofNullable(formerStreamsLeasesDeletionStrategy.streamIdentifiersForLeaseCleanup()).ifPresent( streamIdentifiers -> streamIdentifiers.stream().forEach(streamIdentifier -> enqueueStreamLeaseDeletionOperation.accept(streamIdentifier))); + } else { + // Remove the old/stale streams identified through the new and existing streams list, without + // cleaning up their leases. Disabling deprecated shard sync + lease cleanup through a flag. + Iterator currentSetOfStreamsIter = currentStreamConfigMap.keySet().iterator(); + while (currentSetOfStreamsIter.hasNext()) { + StreamIdentifier streamIdentifier = currentSetOfStreamsIter.next(); + if (!newStreamConfigMap.containsKey(streamIdentifier)) { + if (SHOULD_DO_LEASE_SYNC_FOR_OLD_STREAMS) { + log.info( + "Found old/deleted stream : {}. Triggering shard sync. Removing from tracked active streams." + + streamIdentifier); + ShardSyncTaskManager shardSyncTaskManager = createOrGetShardSyncTaskManager( + currentStreamConfigMap.get(streamIdentifier)); + shardSyncTaskManager.submitShardSyncTask(); + } else { + log.info( + "Found old/deleted stream : {}. Removing from tracked active streams, but not cleaning up leases," + + " as part of this workflow" + streamIdentifier); + } + currentSetOfStreamsIter.remove(); + streamsSynced.add(streamIdentifier); + } + } } // Now let's scan the streamIdentifiersForLeaseCleanup eligible for deferred deletion and delete them. 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 9e15d988..a066ece0 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/coordinator/SchedulerTest.java @@ -468,14 +468,14 @@ public class SchedulerTest { return Duration.ofHours(1); } }); - testMultiStreamStaleStreamsAreNotDeletedImmediately(true); + testMultiStreamStaleStreamsAreNotDeletedImmediately(true, false); } @Test public final void testMultiStreamStaleStreamsAreNotDeletedImmediatelyNoDeletionStrategy() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy()); - testMultiStreamStaleStreamsAreNotDeletedImmediately(false); + testMultiStreamStaleStreamsAreNotDeletedImmediately(false, true); } @Test @@ -490,7 +490,7 @@ public class SchedulerTest { return Duration.ofHours(1); } }); - testMultiStreamStaleStreamsAreNotDeletedImmediately(false); + testMultiStreamStaleStreamsAreNotDeletedImmediately(false, false); } @Test @@ -507,10 +507,11 @@ public class SchedulerTest { return Duration.ofHours(1); } }); - testMultiStreamStaleStreamsAreNotDeletedImmediately(true); + testMultiStreamStaleStreamsAreNotDeletedImmediately(true, false); } - private final void testMultiStreamStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion) + private final void testMultiStreamStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion, + boolean onlyStreamsDeletionNotLeases) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -533,8 +534,9 @@ public class SchedulerTest { Set expectedPendingStreams = IntStream.range(1, 3).mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))).collect( Collectors.toCollection(HashSet::new)); - Assert.assertEquals(Sets.newHashSet(), syncedStreams); - Assert.assertEquals(Sets.newHashSet(streamConfigList1), + Set expectedSyncedStreams = onlyStreamsDeletionNotLeases ? expectedPendingStreams : Sets.newHashSet(); + Assert.assertEquals(expectedSyncedStreams, syncedStreams); + Assert.assertEquals(Sets.newHashSet(onlyStreamsDeletionNotLeases ? streamConfigList2 : streamConfigList1), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams : Sets.newHashSet(), scheduler.staleStreamDeletionMap().keySet()); @@ -625,14 +627,14 @@ public class SchedulerTest { return Duration.ofHours(1); } }); - testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false); } @Test public final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediatelyWithNoDeletionStrategy() throws DependencyException, ProvisionedThroughputException, InvalidStateException { when(multiStreamTracker.formerStreamsLeasesDeletionStrategy()).thenReturn(new NoLeaseDeletionStrategy()); - testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false, true); } @Test @@ -647,7 +649,7 @@ public class SchedulerTest { return Duration.ofHours(1); } }); - testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(false, false); } @Test @@ -665,10 +667,11 @@ public class SchedulerTest { return Duration.ofHours(1); } }); - testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true); + testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(true, false); } - private final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion) + private final void testMultiStreamNewStreamsAreSyncedAndStaleStreamsAreNotDeletedImmediately(boolean expectPendingStreamsForDeletion, + boolean onlyStreamsNoLeasesDeletion) throws DependencyException, ProvisionedThroughputException, InvalidStateException { List streamConfigList1 = IntStream.range(1, 5).mapToObj(streamId -> new StreamConfig( StreamIdentifier.multiStreamInstance( @@ -687,20 +690,39 @@ public class SchedulerTest { metricsConfig, processorConfig, retrievalConfig)); when(scheduler.shouldSyncStreamsNow()).thenReturn(true); Set syncedStreams = scheduler.checkAndSyncStreamShardsAndLeases(); - Set expectedSyncedStreams = IntStream.range(5, 7) - .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) - .collect(Collectors.toCollection(HashSet::new)); + Set expectedSyncedStreams; Set expectedPendingStreams = IntStream.range(1, 3) .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) .collect(Collectors.toCollection(HashSet::new)); + + if(onlyStreamsNoLeasesDeletion) { + expectedSyncedStreams = IntStream.concat(IntStream.range(1, 3), IntStream.range(5, 7)) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + } else { + expectedSyncedStreams = IntStream.range(5, 7) + .mapToObj(streamId -> StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345))) + .collect(Collectors.toCollection(HashSet::new)); + } + Assert.assertEquals(expectedSyncedStreams, syncedStreams); - List expectedCurrentStreamConfigs = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig( - StreamIdentifier.multiStreamInstance( - Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), - InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) - .collect(Collectors.toCollection(LinkedList::new)); + List expectedCurrentStreamConfigs; + if(onlyStreamsNoLeasesDeletion) { + expectedCurrentStreamConfigs = IntStream.range(3, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + } else { + expectedCurrentStreamConfigs = IntStream.range(1, 7).mapToObj(streamId -> new StreamConfig( + StreamIdentifier.multiStreamInstance( + Joiner.on(":").join(streamId * 111111111, "multiStreamTest-" + streamId, streamId * 12345)), + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST))) + .collect(Collectors.toCollection(LinkedList::new)); + } Assert.assertEquals(Sets.newHashSet(expectedCurrentStreamConfigs), Sets.newHashSet(scheduler.currentStreamConfigMap().values())); Assert.assertEquals(expectPendingStreamsForDeletion ? expectedPendingStreams: Sets.newHashSet(), From 8d40e5bdd40c6738b633b7451944c0dd4d08c904 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 17 Jun 2020 10:49:57 -0700 Subject: [PATCH 121/159] Skipping ddb scan and shard sync for streams that has no shards returned from service --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 3 +++ 1 file changed, 3 insertions(+) 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 5c18ee85..b3cfdb56 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 @@ -129,6 +129,9 @@ public class HierarchicalShardSyncer { if (!CollectionUtils.isNullOrEmpty(latestShards)) { log.debug("{} - Num shards: {}", streamIdentifier, latestShards.size()); + } else { + log.warn("Skipping shard sync for {} as no shards found from service.", streamIdentifier); + return; } final Map shardIdToShardMap = constructShardIdToShardMap(latestShards); From 27613763acec74c0a3af08331065d139b96dbe2f Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 17 Jun 2020 11:31:22 -0700 Subject: [PATCH 122/159] search child shards in DDB instead of in local cache --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 9743704d..d8496a91 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 @@ -246,7 +246,7 @@ public class ShutdownTask implements ConsumerTask { throws DependencyException, InvalidStateException, ProvisionedThroughputException { for(ChildShard childShard : childShards) { final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId()); - if(leaseCoordinator.getCurrentlyHeldLease(leaseKey) == null) { + if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) { final Lease leaseToCreate = hierarchicalShardSyncer.createLeaseForChildShard(childShard, shardDetector.streamIdentifier()); leaseCoordinator.leaseRefresher().createLeaseIfNotExists(leaseToCreate); log.info("Shard {}: Created child shard lease: {}", shardInfo.shardId(), leaseToCreate.leaseKey()); From 0246e1e852540ef77b6b623e1db209d90ad29561 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 22 Jun 2020 14:30:28 -0700 Subject: [PATCH 123/159] Only making leader to do the stream sync --- .../software/amazon/kinesis/coordinator/Scheduler.java | 8 +++++++- .../software/amazon/kinesis/lifecycle/ShutdownTask.java | 4 ++-- 2 files changed, 9 insertions(+), 3 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 6a87db23..b13fc6b1 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 @@ -423,7 +423,9 @@ public class Scheduler implements Runnable { cleanupShardConsumers(assignedShards); // check for new streams and sync with the scheduler state - checkAndSyncStreamShardsAndLeases(); + if (isLeader()) { + checkAndSyncStreamShardsAndLeases(); + } logExecutorState(); slog.info("Sleeping ..."); @@ -440,6 +442,10 @@ public class Scheduler implements Runnable { slog.resetInfoLogging(); } + private boolean isLeader() { + return leaderDecider.isLeader(leaseManagementConfig.workerIdentifier()); + } + /** * Note: This method has package level access solely for testing purposes. 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 d8496a91..5e11fddf 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 @@ -113,8 +113,8 @@ public class ShutdownTask implements ConsumerTask { try { try { - log.debug("Invoking shutdown() for shard {}, concurrencyToken {}. Shutdown reason: {}", - leaseKeyProvider.apply(shardInfo), shardInfo.concurrencyToken(), reason); + log.debug("Invoking shutdown() for shard {} with child shards {} , concurrencyToken {}. Shutdown reason: {}", + leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason); final long startTime = System.currentTimeMillis(); if (reason == ShutdownReason.SHARD_END) { From d4f3c0b14a0f4af304dca832dc00648bef81213c Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 22 Jun 2020 23:23:41 -0700 Subject: [PATCH 124/159] Fixing Prefetch publisher cache restart issue --- .../retrieval/polling/PrefetchRecordsPublisher.java | 8 ++++---- .../retrieval/polling/PrefetchRecordsPublisherTest.java | 8 ++++++++ 2 files changed, 12 insertions(+), 4 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 c80c9860..d92c60af 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 @@ -230,12 +230,12 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { if (executorService.isShutdown()) { throw new IllegalStateException("ExecutorService has been shutdown."); } - - publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended); - if (!started) { - log.info("{} : Starting prefetching thread.", streamAndShardId); + publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended); + log.info("{} : Starting prefetching thread and initializing publisher session.", streamAndShardId); executorService.execute(defaultGetRecordsCacheDaemon); + } else { + log.info("{} : Skipping publisher start as it was already started.", streamAndShardId); } started = true; } 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 281d738c..20a37c26 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 @@ -145,6 +145,14 @@ public class PrefetchRecordsPublisherTest { when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResponse); } + @Test + public void testDataFetcherIsNotReInitializedOnMultipleCacheStarts() { + getRecordsCache.start(sequenceNumber, initialPosition); + getRecordsCache.start(sequenceNumber, initialPosition); + getRecordsCache.start(sequenceNumber, initialPosition); + verify(dataFetcher, times(1)).initialize(any(ExtendedSequenceNumber.class), any()); + } + @Test public void testGetRecords() { record = Record.builder().data(createByteBufferWithSize(SIZE_512_KB)).build(); From ef39ecd0df2532bf5772e2d67df195b83b576df9 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 22 Jun 2020 23:37:38 -0700 Subject: [PATCH 125/159] Log changes --- .../kinesis/retrieval/polling/PrefetchRecordsPublisher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d92c60af..ef752f1b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisher.java @@ -231,8 +231,8 @@ public class PrefetchRecordsPublisher implements RecordsPublisher { throw new IllegalStateException("ExecutorService has been shutdown."); } if (!started) { + log.info("{} : Starting Prefetching thread and initializing publisher session.", streamAndShardId); publisherSession.init(extendedSequenceNumber, initialPositionInStreamExtended); - log.info("{} : Starting prefetching thread and initializing publisher session.", streamAndShardId); executorService.execute(defaultGetRecordsCacheDaemon); } else { log.info("{} : Skipping publisher start as it was already started.", streamAndShardId); From 67ce7a783d58bb451530bac96f922086118ad92c Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 23 Jun 2020 03:03:42 -0700 Subject: [PATCH 126/159] Fixing lease cleanup issues with multistreaming --- .../amazon/kinesis/coordinator/Scheduler.java | 5 +- .../kinesis/leases/LeaseCleanupManager.java | 79 +++++++++++-------- .../kinesis/lifecycle/ShutdownTask.java | 53 ++----------- 3 files changed, 57 insertions(+), 80 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 b13fc6b1..db9cc145 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 @@ -532,15 +532,14 @@ 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(); } else { log.info( "Found old/deleted stream : {}. Removing from tracked active streams, but not cleaning up leases," - + " as part of this workflow" + streamIdentifier); + + " as part of this workflow", streamIdentifier); } currentSetOfStreamsIter.remove(); streamsSynced.add(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 0a2b65eb..1152d157 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 @@ -114,20 +114,16 @@ public class LeaseCleanupManager { log.warn("Cannot enqueue lease {} for deferred deletion - instance doesn't hold the lease for that shard.", lease.leaseKey()); } else { - //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597. - if (!deletionQueue.contains(leasePendingDeletion)) { - log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey()); - if (!deletionQueue.add(leasePendingDeletion)) { - log.warn("Unable to enqueue lease {} for deletion.", lease.leaseKey()); - } - } else { - log.warn("Lease {} is already pending deletion, not enqueueing for deletion.", lease.leaseKey()); + log.debug("Enqueuing lease {} for deferred deletion.", lease.leaseKey()); + if (!deletionQueue.add(leasePendingDeletion)) { + log.warn("Unable to enqueue lease {} for deletion.", lease.leaseKey()); } } } /** * Check if lease was already enqueued for deletion. + * //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597. * @param leasePendingDeletion * @return true if enqueued for deletion; false otherwise. */ @@ -168,26 +164,39 @@ public class LeaseCleanupManager { try { if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) { - Set childShardKeys = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey()).childShardIds(); - if (CollectionUtils.isNullOrEmpty(childShardKeys)) { - try { - childShardKeys = getChildShardsFromService(shardInfo, streamIdentifier); + final Lease leaseFromDDB = leaseCoordinator.leaseRefresher().getLease(lease.leaseKey()); + if(leaseFromDDB != null) { + Set childShardKeys = leaseFromDDB.childShardIds(); + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + try { + childShardKeys = getChildShardsFromService(shardInfo, streamIdentifier); - if (CollectionUtils.isNullOrEmpty(childShardKeys)) { - log.error("No child shards returned from service for shard {} for {}.", shardInfo.shardId(), streamIdentifier.streamName()); - } else { - wereChildShardsPresent = true; - updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + log.error( + "No child shards returned from service for shard {} for {} while cleaning up lease.", + shardInfo.shardId(), streamIdentifier.streamName()); + } else { + wereChildShardsPresent = true; + updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + } + } catch (ExecutionException e) { + throw exceptionManager.apply(e.getCause()); + } finally { + alreadyCheckedForGarbageCollection = true; } - } catch (ExecutionException e) { - throw exceptionManager.apply(e.getCause()); - } finally { - alreadyCheckedForGarbageCollection = true; + } else { + wereChildShardsPresent = true; + } + try { + cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys); + } catch (Exception e) { + // Suppressing the exception here, so that we can attempt for garbage cleanup. + log.warn("Unable to cleanup lease for shard {} in {}", shardInfo.shardId(), streamIdentifier.streamName(), e); } } else { - wereChildShardsPresent = true; + log.info("Lease not present in lease table while cleaning the shard {} of {}", shardInfo.shardId(), streamIdentifier.streamName()); + cleanedUpCompletedLease = true; } - cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, shardInfo, childShardKeys); } if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) { @@ -256,20 +265,24 @@ public class LeaseCleanupManager { // 2. Its parent shard lease(s) have already been deleted. private boolean cleanupLeaseForCompletedShard(Lease lease, ShardInfo shardInfo, Set childShardKeys) throws DependencyException, ProvisionedThroughputException, InvalidStateException, IllegalStateException { - final Set processedChildShardLeases = new HashSet<>(); + final Set processedChildShardLeaseKeys = new HashSet<>(); + final Set childShardLeaseKeys = childShardKeys.stream().map(ck -> ShardInfo.getLeaseKey(shardInfo, ck)) + .collect(Collectors.toSet()); - for (String childShardKey : childShardKeys) { - final Lease childShardLease = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(childShardKey)).orElseThrow( - () -> new IllegalStateException("Child lease " + childShardKey + " for completed shard not found in " + - "lease table - not cleaning up lease " + lease)); + for (String childShardLeaseKey : childShardLeaseKeys) { + final Lease childShardLease = Optional.ofNullable( + leaseCoordinator.leaseRefresher().getLease(childShardLeaseKey)) + .orElseThrow(() -> new IllegalStateException( + "Child lease " + childShardLeaseKey + " for completed shard not found in " + + "lease table - not cleaning up lease " + lease)); - if (!childShardLease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) - && !childShardLease.checkpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) { - processedChildShardLeases.add(childShardLease.leaseKey()); + if (!childShardLease.checkpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) && !childShardLease + .checkpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) { + processedChildShardLeaseKeys.add(childShardLease.leaseKey()); } } - if (!allParentShardLeasesDeleted(lease, shardInfo) || !Objects.equals(childShardKeys, processedChildShardLeases)) { + if (!allParentShardLeasesDeleted(lease, shardInfo) || !Objects.equals(childShardLeaseKeys, processedChildShardLeaseKeys)) { return false; } @@ -320,6 +333,8 @@ public class LeaseCleanupManager { if (leaseCleanupResult.leaseCleanedUp()) { log.debug("Successfully cleaned up lease {} for {}", leaseKey, streamIdentifier); deletionSucceeded = true; + } else { + log.warn("Unable to clean up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult); } } catch (Exception e) { log.error("Failed to cleanup lease {} for {}. Will re-enqueue for deletion and retry on next " + 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 5e11fddf..fc206bea 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 @@ -24,6 +24,7 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.utils.CollectionUtils; +import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; @@ -101,7 +102,7 @@ public class ShutdownTask implements ConsumerTask { /* * Invokes ShardRecordProcessor shutdown() API. * (non-Javadoc) - * + * * @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerTask#call() */ @Override @@ -113,7 +114,7 @@ public class ShutdownTask implements ConsumerTask { try { try { - log.debug("Invoking shutdown() for shard {} with child shards {} , concurrencyToken {}. Shutdown reason: {}", + log.debug("Invoking shutdown() for shard {} with childShards {}, concurrencyToken {}. Shutdown reason: {}", leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason); final long startTime = System.currentTimeMillis(); @@ -124,34 +125,19 @@ 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. final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + Validate.validState(currentShardLease != null, + "%s : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.", + leaseKeyProvider.apply(shardInfo)); final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); if (!CollectionUtils.isNullOrEmpty(childShards)) { createLeasesForChildShardsIfNotExist(); updateLeaseWithChildShards(currentShardLease); - // Attempt to do shard checkpointing and throw on exception. + } + if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { attemptShardEndCheckpointing(scope, startTime); - // Enqueue completed shard for deletion. leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); - - } else { - // This might be a case of ResourceNotFound from Service. Directly validate and delete lease, if required. - // If already enqueued for deletion as part of this worker, do not attempt to shard end checkpoint - // or lease cleanup. Else try to shard end checkpoint and cleanup the lease if the shard is a - // garbage shard. - if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { - try { - // Do a best effort shard end checkpointing, before attempting to cleanup the lease, - // in the case of RNF Exception. - attemptShardEndCheckpointing(scope, startTime); - } finally { - // Attempt to garbage collect if this shard is no longer associated with the stream. - // If we don't want to cleanup the garbage shard without successful shard end - // checkpointing, remove the try finally construct and only execute the methods. - attemptGarbageCollectionOfLeaseAndEnqueueOnFailure(leasePendingDeletion, currentShardLease); - } - } } } else { throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); @@ -195,29 +181,6 @@ public class ShutdownTask implements ConsumerTask { } } - private void attemptGarbageCollectionOfLeaseAndEnqueueOnFailure(LeasePendingDeletion leasePendingDeletion, Lease currentShardLease) { - final LeaseCleanupManager.LeaseCleanupResult leaseCleanupResult; - try { - leaseCleanupResult = leaseCleanupManager - .cleanupLease(leasePendingDeletion, false, true); - if (leaseCleanupResult.leaseCleanedUp()) { - log.info("Cleaned up garbage lease {} for {}. Details : {}", - currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); - } else { - log.error("Unable to cleanup potential garbage lease {} for {}. Details : {} ", - currentShardLease.leaseKey(), streamIdentifier, leaseCleanupResult); - // If we are unable to delete this lease and the reason being RNF, then enqueue it - // for deletion, so that we don't end up consuming service TPS on any bugs. - if (leaseCleanupResult.wasResourceNotFound()) { - leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); - } - } - } catch (Exception e) { - log.error("Unable to cleanup potential garbage lease {} for {}", currentShardLease.leaseKey(), - streamIdentifier, e); - } - } - private void applicationCheckpointAndVerification() { recordProcessorCheckpointer .sequenceNumberAtShardEnd(recordProcessorCheckpointer.largestPermittedCheckpointValue()); From 60af78f7cbd8fa1adcc5ca2f141c2575b9d26e75 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 23 Jun 2020 14:49:15 -0700 Subject: [PATCH 127/159] Adding unit test case for validating internal state on initial prefetcher failures --- .../polling/PrefetchRecordsPublisherTest.java | 35 +++++++++++++++++++ .../amazon/kinesis/utils/BlockingUtils.java | 13 +++++++ 2 files changed, 48 insertions(+) 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 20a37c26..f12e2310 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/retrieval/polling/PrefetchRecordsPublisherTest.java @@ -31,10 +31,12 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static software.amazon.kinesis.utils.BlockingUtils.blockUntilConditionSatisfied; import static software.amazon.kinesis.utils.BlockingUtils.blockUntilRecordsAvailable; import static software.amazon.kinesis.utils.ProcessRecordsInputMatcher.eqProcessRecordsInput; @@ -47,6 +49,7 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -57,6 +60,7 @@ import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; @@ -153,6 +157,37 @@ public class PrefetchRecordsPublisherTest { verify(dataFetcher, times(1)).initialize(any(ExtendedSequenceNumber.class), any()); } + @Test + public void testPrefetchPublisherInternalStateNotModifiedWhenPrefetcherThreadStartFails() { + doThrow(new RejectedExecutionException()).doThrow(new RejectedExecutionException()).doCallRealMethod() + .when(executorService).execute(any()); + // Initialize try 1 + tryPrefetchCacheStart(); + blockUntilConditionSatisfied(() -> getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == MAX_SIZE, 300); + verifyInternalState(0); + // Initialize try 2 + tryPrefetchCacheStart(); + blockUntilConditionSatisfied(() -> getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == MAX_SIZE, 300); + verifyInternalState(0); + // Initialize try 3 + tryPrefetchCacheStart(); + blockUntilConditionSatisfied(() -> getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == MAX_SIZE, 300); + verifyInternalState(MAX_SIZE); + verify(dataFetcher, times(3)).initialize(any(ExtendedSequenceNumber.class), any()); + } + + private void tryPrefetchCacheStart() { + try { + getRecordsCache.start(sequenceNumber, initialPosition); + } catch (Exception e) { + // suppress exception + } + } + + private void verifyInternalState(int queueSize) { + Assert.assertTrue(getRecordsCache.getPublisherSession().prefetchRecordsQueue().size() == queueSize); + } + @Test public void testGetRecords() { record = Record.builder().data(createByteBufferWithSize(SIZE_512_KB)).build(); diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java index fa10557f..0d68e51b 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/utils/BlockingUtils.java @@ -35,4 +35,17 @@ public class BlockingUtils { throw new RuntimeException("No records found"); } } + + public static boolean blockUntilConditionSatisfied(Supplier conditionSupplier, long timeoutMillis) { + while(!conditionSupplier.get() && timeoutMillis > 0 ) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + timeoutMillis -= 100; + } + return conditionSupplier.get(); + } + } From a2b60193369f3505060cc32f384b3fb6bd9f2c6b Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 24 Jun 2020 12:39:18 -0700 Subject: [PATCH 128/159] Changes to enqueue lease for deletion in potential garbage cleanup scenario. --- .../kinesis/leases/LeaseCleanupManager.java | 3 +- .../kinesis/lifecycle/ShutdownTask.java | 17 ++++++-- .../leases/LeaseCleanupManagerTest.java | 43 ------------------- 3 files changed, 16 insertions(+), 47 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 1152d157..7b306a68 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 @@ -105,7 +105,8 @@ public class LeaseCleanupManager { } /** - * Enqueues a lease for deletion. + * Enqueues a lease for deletion without check for duplicate entry. Use {@link #isEnqueuedForDeletion} + * for checking the duplicate entries. * @param leasePendingDeletion */ public void enqueueForDeletion(LeasePendingDeletion leasePendingDeletion) { 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 fc206bea..d6282cff 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 @@ -136,8 +136,18 @@ public class ShutdownTask implements ConsumerTask { updateLeaseWithChildShards(currentShardLease); } if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { - attemptShardEndCheckpointing(scope, startTime); - leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + boolean isSuccess = false; + try { + isSuccess = attemptShardEndCheckpointing(scope, startTime); + } finally { + // Check if either the shard end ddb persist is successful or + // if childshards is empty. When child shards is empty then either it is due to + // completed shard being reprocessed or we got RNF from service. + // For these cases enqueue the lease for deletion. + if (isSuccess || !CollectionUtils.isNullOrEmpty(childShards)) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + } + } } } else { throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); @@ -169,7 +179,7 @@ public class ShutdownTask implements ConsumerTask { return new TaskResult(exception); } - private void attemptShardEndCheckpointing(MetricsScope scope, long startTime) + private boolean attemptShardEndCheckpointing(MetricsScope scope, long startTime) throws DependencyException, ProvisionedThroughputException, InvalidStateException, CustomerApplicationException { final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) @@ -179,6 +189,7 @@ public class ShutdownTask implements ConsumerTask { // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded. throwOnApplicationException(() -> applicationCheckpointAndVerification(), scope, startTime); } + return true; } private void applicationCheckpointAndVerification() { 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 eb06a4a0..d02ced04 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 @@ -225,49 +225,6 @@ public class LeaseCleanupManagerTest { verify(leaseRefresher, times(1)).deleteLease(heldLease); } - /** - * Tests that if a lease deletion fails, it's re-enqueued for deletion. - */ - @Test - public final void testFailedDeletionsReEnqueued() throws Exception { - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - - final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); - - when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); - when(kinesis.getShardIterator(any(GetShardIteratorRequest.class))).thenThrow(Exception.class); - - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); - Assert.assertEquals(1, leaseCleanupManager.leasesPendingDeletion()); - } - - /** - * Tests duplicate leases are not enqueued for deletion. - */ - @Test - public final void testNoDuplicateLeasesEnqueued() { - // Disable lease cleanup so that the queue isn't drained while the test is running. - cleanupLeasesOfCompletedShards = false; - leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, - NULL_METRICS_FACTORY, maxFutureWait, deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, - completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); - shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), - ExtendedSequenceNumber.LATEST); - final Lease heldLease = LeaseHelper.createLease(shardInfo.shardId(), "leaseOwner", Collections.singleton("parentShardId")); - - when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); - - // Enqueue the same lease twice. - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); - Assert.assertEquals(1, leaseCleanupManager.leasesPendingDeletion()); - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); - Assert.assertEquals(1, leaseCleanupManager.leasesPendingDeletion()); - } - private final void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, ExtendedSequenceNumber extendedSequenceNumber, int expectedDeletedLeases) throws Exception { From bd6a7d8b906e1038a5a44a9cdb54603ff8fc6203 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 24 Jun 2020 12:43:55 -0700 Subject: [PATCH 129/159] Logging to track pending leases to cleanup --- .../amazon/kinesis/leases/LeaseCleanupManager.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 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 7b306a68..119402cb 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 @@ -136,7 +136,7 @@ public class LeaseCleanupManager { * Returns how many leases are currently waiting in the queue pending deletion. * @return number of leases pending deletion. */ - public int leasesPendingDeletion() { + private int leasesPendingDeletion() { return deletionQueue.size(); } @@ -311,6 +311,7 @@ public class LeaseCleanupManager { @VisibleForTesting void cleanupLeases() { + log.info("Number of pending leases to clean before the scan : {}", leasesPendingDeletion()); if (deletionQueue.isEmpty()) { log.debug("No leases pending deletion."); } else if (timeToCheckForCompletedShard() | timeToCheckForGarbageShard()) { @@ -341,24 +342,22 @@ public class LeaseCleanupManager { log.error("Failed to cleanup lease {} for {}. Will re-enqueue for deletion and retry on next " + "scheduled execution.", leaseKey, streamIdentifier, e); } - if (!deletionSucceeded) { log.debug("Did not cleanup lease {} for {}. Re-enqueueing for deletion.", leaseKey, streamIdentifier); failedDeletions.add(leasePendingDeletion); } } - if (completedLeaseCleanedUp) { log.debug("At least one completed lease was cleaned up - restarting interval"); completedLeaseStopwatch.reset().start(); } - if (garbageLeaseCleanedUp) { log.debug("At least one garbage lease was cleaned up - restarting interval"); garbageLeaseStopwatch.reset().start(); } - deletionQueue.addAll(failedDeletions); + + log.info("Number of pending leases to clean after the scan : {}", leasesPendingDeletion()); } } From b636edd007ed5a37e93d403ca063a252b3c25dbe Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 24 Jun 2020 13:07:42 -0700 Subject: [PATCH 130/159] ShutdownTask bug fix --- .../software/amazon/kinesis/lifecycle/ShutdownTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 5920646c..d5af6627 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 @@ -153,7 +153,7 @@ public class ShutdownTaskTest { final TaskResult result = task.call(); assertNotNull(result.getException()); - assertTrue(result.getException() instanceof KinesisClientLibIOException); + assertTrue(result.getException() instanceof IllegalStateException); verify(recordsPublisher, never()).shutdown(); verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); From 950faf74756b5f048841d8c2fcbb4e285865303d Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 24 Jun 2020 14:34:01 -0700 Subject: [PATCH 131/159] Fixing condition check --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d6282cff..91ea125b 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 @@ -144,7 +144,7 @@ public class ShutdownTask implements ConsumerTask { // if childshards is empty. When child shards is empty then either it is due to // completed shard being reprocessed or we got RNF from service. // For these cases enqueue the lease for deletion. - if (isSuccess || !CollectionUtils.isNullOrEmpty(childShards)) { + if (isSuccess || CollectionUtils.isNullOrEmpty(childShards)) { leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); } } From f693311ac8a30de181f2f1cb298616a058da30d6 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+jushkem@users.noreply.github.com> Date: Wed, 1 Jul 2020 19:01:13 -0400 Subject: [PATCH 132/159] Fixing bug for multistream not using list shards with filter. (#66) Fixing bug for mulitstreaming not using list shards with filter. --- .../kinesis/leases/HierarchicalShardSyncer.java | 2 -- .../amazon/kinesis/leases/LeaseRefresher.java | 13 +++++++++++++ .../amazon/kinesis/leases/ShardSyncTask.java | 2 +- .../leases/dynamodb/DynamoDBLeaseRefresher.java | 8 +++++++- .../leases/ExceptionThrowingLeaseRefresher.java | 5 +++++ .../kinesis/leases/HierarchicalShardSyncerTest.java | 12 +++++++----- 6 files changed, 33 insertions(+), 9 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 b3cfdb56..c67b3d80 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 @@ -125,8 +125,6 @@ public class HierarchicalShardSyncer { 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 { 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 b7f38a4e..8dc46364 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 @@ -217,6 +217,19 @@ public interface LeaseRefresher { */ boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException; + /** + * Check (synchronously) if there are any leases in the lease table for a given stream identifier. + * + * @param streamIdentifier for multi-stream mode. Can be null. + * @return true if there are no leases in the lease table + * + * @throws DependencyException if DynamoDB scan fails in an unexpected way + * @throws InvalidStateException if lease table does not exist + * @throws ProvisionedThroughputException if DynamoDB scan fails due to lack of capacity + */ + boolean isLeaseTableEmptyForStreamIdentifier(StreamIdentifier streamIdentifier) throws DependencyException, + InvalidStateException, ProvisionedThroughputException; + /** * Gets the current checkpoint of the shard. This is useful in the resharding use case * where we will wait for the parent shard to complete before starting on the records from a child shard. 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 820d4528..12e05550 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 @@ -69,7 +69,7 @@ public class ShardSyncTask implements ConsumerTask { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, scope, ignoreUnexpectedChildShards, - leaseRefresher.isLeaseTableEmpty()); + leaseRefresher.isLeaseTableEmptyForStreamIdentifier(shardDetector.streamIdentifier())); if (shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); 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 df5746a2..913c7331 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 @@ -292,7 +292,13 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(1, 1, null).isEmpty(); + return isLeaseTableEmptyForStreamIdentifier(null); + } + + @Override + public boolean isLeaseTableEmptyForStreamIdentifier(StreamIdentifier streamIdentifier) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + return list(1, 1, streamIdentifier).isEmpty(); } /** 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 81a49839..557e2b38 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 @@ -215,6 +215,11 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { return false; } + @Override + public boolean isLeaseTableEmptyForStreamIdentifier(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + return false; + } + @Override public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException { 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..f4c1dd2c 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 @@ -335,7 +335,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier( + StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -417,8 +418,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - latestShards, false, SCOPE, - dynamoDBLeaseRefresher.isLeaseTableEmpty()); + latestShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier( + StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -682,7 +683,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, - dynamoDBLeaseRefresher.isLeaseTableEmpty()); + dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier(StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -762,7 +763,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); + SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier( + StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); From 74ffd4060c48ea68bc374f47bc557fb7738f9c3a Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Wed, 1 Jul 2020 19:07:50 -0400 Subject: [PATCH 133/159] Revert "Fixing bug for multistream not using list shards with filter. (#66)" This reverts commit f693311ac8a30de181f2f1cb298616a058da30d6. --- .../kinesis/leases/HierarchicalShardSyncer.java | 2 ++ .../amazon/kinesis/leases/LeaseRefresher.java | 13 ------------- .../amazon/kinesis/leases/ShardSyncTask.java | 2 +- .../leases/dynamodb/DynamoDBLeaseRefresher.java | 8 +------- .../leases/ExceptionThrowingLeaseRefresher.java | 5 ----- .../kinesis/leases/HierarchicalShardSyncerTest.java | 12 +++++------- 6 files changed, 9 insertions(+), 33 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 c67b3d80..b3cfdb56 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 @@ -125,6 +125,8 @@ public class HierarchicalShardSyncer { 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 { 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 8dc46364..b7f38a4e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseRefresher.java @@ -217,19 +217,6 @@ public interface LeaseRefresher { */ boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException; - /** - * Check (synchronously) if there are any leases in the lease table for a given stream identifier. - * - * @param streamIdentifier for multi-stream mode. Can be null. - * @return true if there are no leases in the lease table - * - * @throws DependencyException if DynamoDB scan fails in an unexpected way - * @throws InvalidStateException if lease table does not exist - * @throws ProvisionedThroughputException if DynamoDB scan fails due to lack of capacity - */ - boolean isLeaseTableEmptyForStreamIdentifier(StreamIdentifier streamIdentifier) throws DependencyException, - InvalidStateException, ProvisionedThroughputException; - /** * Gets the current checkpoint of the shard. This is useful in the resharding use case * where we will wait for the parent shard to complete before starting on the records from a child shard. 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 12e05550..820d4528 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 @@ -69,7 +69,7 @@ public class ShardSyncTask implements ConsumerTask { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, scope, ignoreUnexpectedChildShards, - leaseRefresher.isLeaseTableEmptyForStreamIdentifier(shardDetector.streamIdentifier())); + leaseRefresher.isLeaseTableEmpty()); if (shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); 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 913c7331..df5746a2 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 @@ -292,13 +292,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return isLeaseTableEmptyForStreamIdentifier(null); - } - - @Override - public boolean isLeaseTableEmptyForStreamIdentifier(StreamIdentifier streamIdentifier) - throws DependencyException, ProvisionedThroughputException, InvalidStateException { - return list(1, 1, streamIdentifier).isEmpty(); + return list(1, 1, null).isEmpty(); } /** 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 557e2b38..81a49839 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/ExceptionThrowingLeaseRefresher.java @@ -215,11 +215,6 @@ public class ExceptionThrowingLeaseRefresher implements LeaseRefresher { return false; } - @Override - public boolean isLeaseTableEmptyForStreamIdentifier(StreamIdentifier streamIdentifier) throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return false; - } - @Override public ExtendedSequenceNumber getCheckpoint(final String leaseKey) throws ProvisionedThroughputException, InvalidStateException, DependencyException { 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 f4c1dd2c..c390987c 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/HierarchicalShardSyncerTest.java @@ -335,8 +335,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier( - StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); + SCOPE, false, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -418,8 +417,8 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - latestShards, false, SCOPE, dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier( - StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); + latestShards, false, SCOPE, + dynamoDBLeaseRefresher.isLeaseTableEmpty()); final Set expectedShardIds = new HashSet<>( toMultiStreamLeaseList(Arrays.asList("shardId-4", "shardId-8", "shardId-9", "shardId-10"))); @@ -683,7 +682,7 @@ public class HierarchicalShardSyncerTest { try { hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_TRIM_HORIZON, SCOPE, false, - dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier(StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); + dynamoDBLeaseRefresher.isLeaseTableEmpty()); } finally { verify(shardDetector).listShards(); verify(dynamoDBLeaseRefresher, never()).listLeases(); @@ -763,8 +762,7 @@ public class HierarchicalShardSyncerTest { setupMultiStream(); hierarchicalShardSyncer .checkAndCreateLeaseForNewShards(shardDetector, dynamoDBLeaseRefresher, INITIAL_POSITION_LATEST, - SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmptyForStreamIdentifier( - StreamIdentifier.multiStreamInstance(STREAM_IDENTIFIER))); + SCOPE, true, dynamoDBLeaseRefresher.isLeaseTableEmpty()); final List leases = leaseCaptor.getAllValues(); final Set leaseKeys = leases.stream().map(Lease::leaseKey).collect(Collectors.toSet()); From b49d8ea8cc0f1025fb4e4191dfb5d2343b1a1208 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 7 Jul 2020 16:02:12 -0700 Subject: [PATCH 134/159] ChildShard validation in Data Fetcher level and error handling in shutdownTask --- .../kinesis/lifecycle/ShutdownTask.java | 50 +++++++++++++------ .../fanout/FanOutRecordsPublisher.java | 16 +++++- .../retrieval/polling/KinesisDataFetcher.java | 15 +++++- .../kinesis/lifecycle/ShutdownTaskTest.java | 17 ++++--- 4 files changed, 72 insertions(+), 26 deletions(-) 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 91ea125b..800aa4c7 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 @@ -35,6 +35,7 @@ import software.amazon.kinesis.leases.LeaseCleanupManager; import software.amazon.kinesis.leases.LeaseCoordinator; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; @@ -116,25 +117,37 @@ public class ShutdownTask implements ConsumerTask { try { log.debug("Invoking shutdown() for shard {} with childShards {}, concurrencyToken {}. Shutdown reason: {}", leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason); + ShutdownReason localReason = reason; final long startTime = System.currentTimeMillis(); - if (reason == ShutdownReason.SHARD_END) { + final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + if (localReason == ShutdownReason.SHARD_END) { // Create new lease for the child shards if they don't exist. // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. // This scenario could happen when customer deletes the stream while leaving the KCL application running. - final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); Validate.validState(currentShardLease != null, - "%s : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.", - leaseKeyProvider.apply(shardInfo)); - final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, - currentShardLease, shardInfo); - - if (!CollectionUtils.isNullOrEmpty(childShards)) { - createLeasesForChildShardsIfNotExist(); - updateLeaseWithChildShards(currentShardLease); + "%s : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.", + leaseKeyProvider.apply(shardInfo)); + try { + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateLeaseWithChildShards(currentShardLease); + } + } catch (InvalidStateException e) { + // If invalidStateException happens, it indicates we are missing childShard related information. + // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry getting + // childShard information in the processTask. + localReason = ShutdownReason.LEASE_LOST; + dropLease(); + log.warn("Shard " + shardInfo.shardId() + ": Exception happened while shutting down shardConsumer with LEASE_LOST reason. " + + "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); } + } + + if (localReason == ShutdownReason.SHARD_END) { + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { boolean isSuccess = false; try { @@ -234,11 +247,7 @@ public class ShutdownTask implements ConsumerTask { final Lease updatedLease = currentLease.copy(); updatedLease.childShardIds(childShardIds); - // TODO : Make changes to use the new leaserefresher#updateLease(Lease lease, UpdateField updateField) - final boolean updateResult = leaseCoordinator.updateLease(updatedLease, UUID.fromString(shardInfo.concurrencyToken()), SHUTDOWN_TASK_OPERATION, leaseKeyProvider.apply(shardInfo)); - if (!updateResult) { - throw new InvalidStateException("Failed to update parent lease with child shard information for shard " + shardInfo.shardId()); - } + leaseCoordinator.leaseRefresher().updateLeaseWithMetaInfo(updatedLease, UpdateField.CHILD_SHARDS); log.info("Shard {}: Updated current lease {} with child shard information: {}", shardInfo.shardId(), currentLease.leaseKey(), childShardIds); } @@ -257,4 +266,15 @@ public class ShutdownTask implements ConsumerTask { return reason; } + private void dropLease() { + Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + if (currentLease == null) { + log.warn("Shard " + shardInfo.shardId() + ": Lease already dropped. Will shutdown the shardConsumer directly."); + return; + } + leaseCoordinator.dropLease(currentLease); + if(currentLease != null) { + log.warn("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); + } + } } 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 38075890..99a7e9a4 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,6 +27,7 @@ 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; @@ -511,8 +512,19 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } private boolean isValidEvent(SubscribeToShardEvent event) { - return event.continuationSequenceNumber() == null ? !CollectionUtils.isNullOrEmpty(event.childShards()) - : event.childShards() != null && event.childShards().isEmpty(); + if (event.continuationSequenceNumber() == null && CollectionUtils.isNullOrEmpty(event.childShards()) || + event.continuationSequenceNumber() != null && !CollectionUtils.isNullOrEmpty(event.childShards())) { + return false; + } + + if(!CollectionUtils.isNullOrEmpty(event.childShards())) { + for (ChildShard childShard : event.childShards()) { + if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + return false; + } + } + } + return true; } private void updateAvailableQueueSpaceAndRequestUpstream(RecordFlow triggeringFlow) { 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 3af3dcf5..c1bb7d7e 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,6 +28,7 @@ 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; @@ -340,8 +341,18 @@ public class KinesisDataFetcher implements DataFetcher { } private boolean isValidResponse(GetRecordsResponse response) { - return response.nextShardIterator() == null ? !CollectionUtils.isNullOrEmpty(response.childShards()) - : response.childShards() != null && response.childShards().isEmpty(); + if (response.nextShardIterator() == null && CollectionUtils.isNullOrEmpty(response.childShards()) || + response.nextShardIterator() != null && !CollectionUtils.isNullOrEmpty(response.childShards())) { + return false; + } + if (!CollectionUtils.isNullOrEmpty(response.childShards())) { + for (ChildShard childShard : response.childShards()) { + if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + return false; + } + } + } + return true; } private AWSExceptionManager createExceptionManager() { 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 d5af6627..792d566a 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 @@ -52,6 +52,7 @@ import software.amazon.kinesis.leases.LeaseRefresher; import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; import software.amazon.kinesis.leases.ShardObjectHelper; +import software.amazon.kinesis.leases.UpdateField; import software.amazon.kinesis.leases.exceptions.CustomerApplicationException; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; @@ -148,16 +149,18 @@ 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(leaseRefresher.createLeaseIfNotExists(Matchers.any(Lease.class))).thenThrow(new KinesisClientLibIOException("KinesisClientLibIOException")); + when(hierarchicalShardSyncer.createLeaseForChildShard(Matchers.any(ChildShard.class), Matchers.any(StreamIdentifier.class))) + .thenThrow(new InvalidStateException("InvalidStateException is thrown")); final TaskResult result = task.call(); - assertNotNull(result.getException()); - assertTrue(result.getException() instanceof IllegalStateException); - verify(recordsPublisher, never()).shutdown(); + assertNull(result.getException()); + verify(recordsPublisher).shutdown(); verify(shardRecordProcessor, never()).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); - verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); - verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); + verify(leaseCoordinator).dropLease(Matchers.any(Lease.class)); } /** @@ -185,7 +188,7 @@ public class ShutdownTaskTest { verify(recordsPublisher).shutdown(); verify(shardRecordProcessor).shardEnded(ShardEndedInput.builder().checkpointer(recordProcessorCheckpointer).build()); verify(shardRecordProcessor, never()).leaseLost(LeaseLostInput.builder().build()); - verify(leaseCoordinator).updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); + 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)); From 347ae9eb68a66542d6d1e25507eebacf6de9bcca Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Tue, 7 Jul 2020 16:58:40 -0700 Subject: [PATCH 135/159] Add very old leases + update lease taker to always emit metrics (#54) * Add very old leases + metrics update * Add conditional emisison for metrics in case of very old leases * move code blocks around to ensure all metrics are emitted --- .../leases/dynamodb/DynamoDBLeaseTaker.java | 67 +++++++++++++------ 1 file changed, 48 insertions(+), 19 deletions(-) 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 41147947..3ec2de22 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java @@ -25,7 +25,7 @@ import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; - +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; import software.amazon.kinesis.annotations.KinesisClientInternalApi; @@ -36,8 +36,8 @@ import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.metrics.MetricsFactory; -import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsLevel; +import software.amazon.kinesis.metrics.MetricsScope; import software.amazon.kinesis.metrics.MetricsUtil; /** @@ -48,6 +48,7 @@ import software.amazon.kinesis.metrics.MetricsUtil; public class DynamoDBLeaseTaker implements LeaseTaker { private static final int TAKE_RETRIES = 3; private static final int SCAN_RETRIES = 1; + private static final int VERY_OLD_LEASE_DURATION_NANOS_MULTIPLIER = 3; // See note on TAKE_LEASES_DIMENSION(Callable) for why we have this callable. private static final Callable SYSTEM_CLOCK_CALLABLE = System::nanoTime; @@ -329,31 +330,39 @@ public class DynamoDBLeaseTaker implements LeaseTaker { Set leasesToTake = new HashSet<>(); final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION); MetricsUtil.addWorkerIdentifier(scope, workerIdentifier); + List veryOldLeases = new ArrayList<>(); + + int numLeases = 0; + int numWorkers = 0; + int numLeasesToReachTarget = 0; + int leaseSpillover = 0; + int veryOldLeaseCount = 0; try { - int numLeases = allLeases.size(); - int numWorkers = leaseCounts.size(); + numLeases = allLeases.size(); + numWorkers = leaseCounts.size(); if (numLeases == 0) { // If there are no leases, I shouldn't try to take any. return leasesToTake; } + int target; if (numWorkers >= numLeases) { // If we have n leases and n or more workers, each worker can have up to 1 lease, including myself. target = 1; } else { - /* - * numWorkers must be < numLeases. - * - * Our target for each worker is numLeases / numWorkers (+1 if numWorkers doesn't evenly divide numLeases) - */ + /* + * numWorkers must be < numLeases. + * + * Our target for each worker is numLeases / numWorkers (+1 if numWorkers doesn't evenly divide numLeases) + */ target = numLeases / numWorkers + (numLeases % numWorkers == 0 ? 0 : 1); // Spill over is the number of leases this worker should have claimed, but did not because it would // exceed the max allowed for this worker. - int leaseSpillover = Math.max(0, target - maxLeasesForWorker); + leaseSpillover = Math.max(0, target - maxLeasesForWorker); if (target > maxLeasesForWorker) { log.warn( "Worker {} target is {} leases and maxLeasesForWorker is {}. Resetting target to {}," @@ -362,11 +371,29 @@ public class DynamoDBLeaseTaker implements LeaseTaker { workerIdentifier, target, maxLeasesForWorker, maxLeasesForWorker, leaseSpillover); target = maxLeasesForWorker; } - scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY); } int myCount = leaseCounts.get(workerIdentifier); - int numLeasesToReachTarget = target - myCount; + numLeasesToReachTarget = target - myCount; + + int currentLeaseCount = leaseCounts.get(workerIdentifier); + // If there are leases that have been expired for an extended period of + // time, take them with priority, disregarding the target (computed + // later) but obeying the maximum limit per worker. + veryOldLeases = allLeases.values().stream() + .filter(lease -> System.nanoTime() - lease.lastCounterIncrementNanos() + > VERY_OLD_LEASE_DURATION_NANOS_MULTIPLIER * leaseDurationNanos) + .collect(Collectors.toList()); + + if (!veryOldLeases.isEmpty()) { + Collections.shuffle(veryOldLeases); + veryOldLeaseCount = Math.max(0, Math.min(maxLeasesForWorker - currentLeaseCount, veryOldLeases.size())); + HashSet result = new HashSet<>(veryOldLeases.subList(0, veryOldLeaseCount)); + if (veryOldLeaseCount > 0) { + log.info("Taking leases that have been expired for a long time: {}", result); + } + return result; + } if (numLeasesToReachTarget <= 0) { // If we don't need anything, return the empty set. @@ -376,7 +403,6 @@ public class DynamoDBLeaseTaker implements LeaseTaker { // Shuffle expiredLeases so workers don't all try to contend for the same leases. Collections.shuffle(expiredLeases); - int originalExpiredLeasesSize = expiredLeases.size(); if (expiredLeases.size() > 0) { // If we have expired leases, get up to leases from expiredLeases for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) { @@ -397,16 +423,19 @@ public class DynamoDBLeaseTaker implements LeaseTaker { log.info( "Worker {} saw {} total leases, {} available leases, {} " + "workers. Target is {} leases, I have {} leases, I will take {} leases", - workerIdentifier, numLeases, originalExpiredLeasesSize, numWorkers, target, myCount, + workerIdentifier, numLeases, expiredLeases.size(), numWorkers, target, myCount, leasesToTake.size()); } - scope.addData("TotalLeases", numLeases, StandardUnit.COUNT, MetricsLevel.DETAILED); - scope.addData("ExpiredLeases", originalExpiredLeasesSize, StandardUnit.COUNT, MetricsLevel.SUMMARY); - scope.addData("NumWorkers", numWorkers, StandardUnit.COUNT, MetricsLevel.SUMMARY); - scope.addData("NeededLeases", numLeasesToReachTarget, StandardUnit.COUNT, MetricsLevel.DETAILED); - scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED); } finally { + scope.addData("ExpiredLeases", expiredLeases.size(), StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("LeaseSpillover", leaseSpillover, StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("LeasesToTake", leasesToTake.size(), StandardUnit.COUNT, MetricsLevel.DETAILED); + scope.addData("NeededLeases", Math.max(numLeasesToReachTarget, 0), StandardUnit.COUNT, MetricsLevel.DETAILED); + scope.addData("NumWorkers", numWorkers, StandardUnit.COUNT, MetricsLevel.SUMMARY); + scope.addData("TotalLeases", numLeases, StandardUnit.COUNT, MetricsLevel.DETAILED); + scope.addData("VeryOldLeases", veryOldLeaseCount, StandardUnit.COUNT, MetricsLevel.SUMMARY); + MetricsUtil.endScope(scope); } From a0094b0df8c9852eb0102c285bae48a18accc16c Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 8 Jul 2020 12:33:16 -0700 Subject: [PATCH 136/159] Addressing comments --- .../kinesis/lifecycle/ShutdownTask.java | 68 ++++++++++--------- .../kinesis/retrieval/DataRetrievalUtil.java | 32 +++++++++ .../fanout/FanOutRecordsPublisher.java | 19 +----- .../retrieval/polling/KinesisDataFetcher.java | 18 +---- 4 files changed, 72 insertions(+), 65 deletions(-) create mode 100644 amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java 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 800aa4c7..0c01f13f 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 @@ -117,34 +117,10 @@ public class ShutdownTask implements ConsumerTask { try { log.debug("Invoking shutdown() for shard {} with childShards {}, concurrencyToken {}. Shutdown reason: {}", leaseKeyProvider.apply(shardInfo), childShards, shardInfo.concurrencyToken(), reason); - ShutdownReason localReason = reason; final long startTime = System.currentTimeMillis(); final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); - if (localReason == ShutdownReason.SHARD_END) { - // Create new lease for the child shards if they don't exist. - // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. - // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. - // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. - // This scenario could happen when customer deletes the stream while leaving the KCL application running. - Validate.validState(currentShardLease != null, - "%s : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.", - leaseKeyProvider.apply(shardInfo)); - try { - if (!CollectionUtils.isNullOrEmpty(childShards)) { - createLeasesForChildShardsIfNotExist(); - updateLeaseWithChildShards(currentShardLease); - } - } catch (InvalidStateException e) { - // If invalidStateException happens, it indicates we are missing childShard related information. - // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry getting - // childShard information in the processTask. - localReason = ShutdownReason.LEASE_LOST; - dropLease(); - log.warn("Shard " + shardInfo.shardId() + ": Exception happened while shutting down shardConsumer with LEASE_LOST reason. " + - "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); - } - } + final ShutdownReason localReason = attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(reason, currentShardLease); if (localReason == ShutdownReason.SHARD_END) { final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); @@ -192,6 +168,36 @@ public class ShutdownTask implements ConsumerTask { return new TaskResult(exception); } + private ShutdownReason attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(ShutdownReason originalReason, Lease currentShardLease) + throws DependencyException, ProvisionedThroughputException { + ShutdownReason localReason = originalReason; + if (originalReason == ShutdownReason.SHARD_END) { + // Create new lease for the child shards if they don't exist. + // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. + // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. + // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. + // This scenario could happen when customer deletes the stream while leaving the KCL application running. + Validate.validState(currentShardLease != null, + "%s : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.", + leaseKeyProvider.apply(shardInfo)); + try { + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateLeaseWithChildShards(currentShardLease); + } + } catch (InvalidStateException e) { + // If InvalidStateException happens, it indicates we are missing childShard related information. + // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry getting + // childShard information in the processTask. + localReason = ShutdownReason.LEASE_LOST; + log.warn("Lease {}: Exception happened while shutting down shardConsumer with SHARD_END reason. " + + "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. Exception: ", currentShardLease.leaseKey(), e); + dropLease(currentShardLease); + } + } + return localReason; + } + private boolean attemptShardEndCheckpointing(MetricsScope scope, long startTime) throws DependencyException, ProvisionedThroughputException, InvalidStateException, CustomerApplicationException { @@ -266,15 +272,13 @@ public class ShutdownTask implements ConsumerTask { return reason; } - private void dropLease() { - Lease currentLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); + private void dropLease(Lease currentLease) { if (currentLease == null) { - log.warn("Shard " + shardInfo.shardId() + ": Lease already dropped. Will shutdown the shardConsumer directly."); + log.warn("Shard {}: Unable to find the lease for shard. Will shutdown the shardConsumer directly.", leaseKeyProvider.apply(shardInfo)); return; - } - leaseCoordinator.dropLease(currentLease); - if(currentLease != null) { - log.warn("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); + } else { + leaseCoordinator.dropLease(currentLease); + log.info("Dropped lease for shutting down ShardConsumer: " + currentLease.leaseKey()); } } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java new file mode 100644 index 00000000..35ad2de6 --- /dev/null +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java @@ -0,0 +1,32 @@ +package software.amazon.kinesis.retrieval; + +import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.utils.CollectionUtils; + +import java.util.List; + +public class DataRetrievalUtil { + + public static boolean isValidResult(String shardEndIndicator, List childShards) { + // shardEndIndicator is nextShardIterator for GetRecordsResponse, and is continuationSequenceNumber for SubscribeToShardEvent + // There are two valid scenarios for the shardEndIndicator and childShards combination. + // 1. ShardEnd scenario: shardEndIndicator should be null and childShards should be a non-empty list. + // 2. Non-ShardEnd scenario: shardEndIndicator should be non-null and childShards should be null or an empty list. + // Otherwise, the retrieval result is invalid. + if (shardEndIndicator == null && CollectionUtils.isNullOrEmpty(childShards) || + shardEndIndicator != null && !CollectionUtils.isNullOrEmpty(childShards)) { + return false; + } + + // For ShardEnd scenario, for each childShard we should validate if parentShards are available. + // Missing parentShards can cause issues with creating leases for childShards during ShardConsumer shutdown. + if (!CollectionUtils.isNullOrEmpty(childShards)) { + for (ChildShard childShard : childShards) { + if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) { + return false; + } + } + } + return true; + } +} diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java index 99a7e9a4..b6299bbf 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 @@ -43,6 +43,7 @@ import software.amazon.kinesis.common.RequestDetails; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.retrieval.BatchUniqueIdentifier; +import software.amazon.kinesis.retrieval.DataRetrievalUtil; import software.amazon.kinesis.retrieval.IteratorBuilder; import software.amazon.kinesis.retrieval.KinesisClientRecord; import software.amazon.kinesis.retrieval.RecordsDeliveryAck; @@ -485,7 +486,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // Since the triggeringFlow is active flow, it will then trigger the handleFlowError call. // Since the exception is not ResourceNotFoundException, it will trigger onError in the ShardConsumerSubscriber. // The ShardConsumerSubscriber will finally cancel the subscription. - if (!isValidEvent(recordBatchEvent)) { + if (!DataRetrievalUtil.isValidResult(recordBatchEvent.continuationSequenceNumber(), recordBatchEvent.childShards())) { throw new InvalidStateException("RecordBatchEvent for flow " + triggeringFlow.toString() + " is invalid." + " event.continuationSequenceNumber: " + recordBatchEvent.continuationSequenceNumber() + ". event.childShards: " + recordBatchEvent.childShards()); @@ -511,22 +512,6 @@ public class FanOutRecordsPublisher implements RecordsPublisher { } } - private boolean isValidEvent(SubscribeToShardEvent event) { - if (event.continuationSequenceNumber() == null && CollectionUtils.isNullOrEmpty(event.childShards()) || - event.continuationSequenceNumber() != null && !CollectionUtils.isNullOrEmpty(event.childShards())) { - return false; - } - - if(!CollectionUtils.isNullOrEmpty(event.childShards())) { - for (ChildShard childShard : event.childShards()) { - if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) { - return false; - } - } - } - return true; - } - private void updateAvailableQueueSpaceAndRequestUpstream(RecordFlow triggeringFlow) { if (availableQueueSpace <= 0) { log.debug( 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 c1bb7d7e..3584dbf8 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 @@ -48,6 +48,7 @@ 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; @@ -290,7 +291,7 @@ public class KinesisDataFetcher implements DataFetcher { public GetRecordsResponse getGetRecordsResponse(GetRecordsRequest request) throws ExecutionException, InterruptedException, TimeoutException { final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request), maxFutureWait); - if (!isValidResponse(response)) { + if (!DataRetrievalUtil.isValidResult(response.nextShardIterator(), response.childShards())) { throw new RetryableRetrievalException("GetRecords response is not valid for shard: " + streamAndShardId + ". nextShardIterator: " + response.nextShardIterator() + ". childShards: " + response.childShards() + ". Will retry GetRecords with the same nextIterator."); @@ -340,21 +341,6 @@ public class KinesisDataFetcher implements DataFetcher { } } - private boolean isValidResponse(GetRecordsResponse response) { - if (response.nextShardIterator() == null && CollectionUtils.isNullOrEmpty(response.childShards()) || - response.nextShardIterator() != null && !CollectionUtils.isNullOrEmpty(response.childShards())) { - return false; - } - if (!CollectionUtils.isNullOrEmpty(response.childShards())) { - for (ChildShard childShard : response.childShards()) { - if (CollectionUtils.isNullOrEmpty(childShard.parentShards())) { - return false; - } - } - } - return true; - } - private AWSExceptionManager createExceptionManager() { final AWSExceptionManager exceptionManager = new AWSExceptionManager(); exceptionManager.add(ResourceNotFoundException.class, t -> t); From 85a5423657ef90516b798dda79b3bfc125ccbe7b Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 14 Jul 2020 15:02:04 -0700 Subject: [PATCH 137/159] Addressing more comments --- .../kinesis/retrieval/DataRetrievalUtil.java | 15 +++++++++++++++ .../retrieval/fanout/FanOutRecordsPublisher.java | 4 ++-- .../retrieval/polling/KinesisDataFetcher.java | 4 +++- 3 files changed, 20 insertions(+), 3 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java index 35ad2de6..ba743e61 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/DataRetrievalUtil.java @@ -1,3 +1,18 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package software.amazon.kinesis.retrieval; import software.amazon.awssdk.services.kinesis.model.ChildShard; 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 b6299bbf..7e8932cf 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/retrieval/fanout/FanOutRecordsPublisher.java @@ -43,7 +43,6 @@ import software.amazon.kinesis.common.RequestDetails; import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; import software.amazon.kinesis.retrieval.BatchUniqueIdentifier; -import software.amazon.kinesis.retrieval.DataRetrievalUtil; import software.amazon.kinesis.retrieval.IteratorBuilder; import software.amazon.kinesis.retrieval.KinesisClientRecord; import software.amazon.kinesis.retrieval.RecordsDeliveryAck; @@ -62,6 +61,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static software.amazon.kinesis.common.DiagnosticUtils.takeDelayedDeliveryActionIfRequired; +import static software.amazon.kinesis.retrieval.DataRetrievalUtil.isValidResult; @Slf4j @KinesisClientInternalApi @@ -486,7 +486,7 @@ public class FanOutRecordsPublisher implements RecordsPublisher { // Since the triggeringFlow is active flow, it will then trigger the handleFlowError call. // Since the exception is not ResourceNotFoundException, it will trigger onError in the ShardConsumerSubscriber. // The ShardConsumerSubscriber will finally cancel the subscription. - if (!DataRetrievalUtil.isValidResult(recordBatchEvent.continuationSequenceNumber(), recordBatchEvent.childShards())) { + if (!isValidResult(recordBatchEvent.continuationSequenceNumber(), recordBatchEvent.childShards())) { throw new InvalidStateException("RecordBatchEvent for flow " + triggeringFlow.toString() + " is invalid." + " event.continuationSequenceNumber: " + recordBatchEvent.continuationSequenceNumber() + ". event.childShards: " + recordBatchEvent.childShards()); 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 3584dbf8..223ab367 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 @@ -54,6 +54,8 @@ import software.amazon.kinesis.retrieval.KinesisDataFetcherProviderConfig; import software.amazon.kinesis.retrieval.RetryableRetrievalException; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; +import static software.amazon.kinesis.retrieval.DataRetrievalUtil.isValidResult; + /** * Used to get data from Amazon Kinesis. Tracks iterator state internally. */ @@ -291,7 +293,7 @@ public class KinesisDataFetcher implements DataFetcher { public GetRecordsResponse getGetRecordsResponse(GetRecordsRequest request) throws ExecutionException, InterruptedException, TimeoutException { final GetRecordsResponse response = FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(request), maxFutureWait); - if (!DataRetrievalUtil.isValidResult(response.nextShardIterator(), response.childShards())) { + if (!isValidResult(response.nextShardIterator(), response.childShards())) { throw new RetryableRetrievalException("GetRecords response is not valid for shard: " + streamAndShardId + ". nextShardIterator: " + response.nextShardIterator() + ". childShards: " + response.childShards() + ". Will retry GetRecords with the same nextIterator."); From ff1bee579100080ac42e5e702d7940573d1484d4 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 14 Jul 2020 01:17:05 -0700 Subject: [PATCH 138/159] Fix for avoiding premature creation of child lease when other parent lease is not created yet. This can happen in the case of fast reshard operations. --- .../kinesis/lifecycle/ShutdownTask.java | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) 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 0c01f13f..d0d9c0c4 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 @@ -17,6 +17,7 @@ package software.amazon.kinesis.lifecycle; import com.google.common.annotations.VisibleForTesting; import java.util.List; +import java.util.Objects; import java.util.Optional; import lombok.NonNull; @@ -237,6 +238,26 @@ public class ShutdownTask implements ConsumerTask { private void createLeasesForChildShardsIfNotExist() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + // For child shard resulted from merge of two parent shards, verify if both the parents are either present or + // not present in the lease table before creating the lease entry. + if (!CollectionUtils.isNullOrEmpty(childShards) && childShards.size() == 1) { + final ChildShard childShard = childShards.get(0); + final List parentLeaseKeys = childShard.parentShards().stream() + .map(parentShardId -> ShardInfo.getLeaseKey(shardInfo, parentShardId)).collect(Collectors.toList()); + if (parentLeaseKeys.size() != 2) { + throw new InvalidStateException("Shard " + shardInfo.shardId() + "'s only child shard " + childShard + + " does not contain other parent information."); + } else { + boolean isValidLeaseTableState = + Objects.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(0))) == Objects + .isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(1))); + if (!isValidLeaseTableState) { + throw new InvalidStateException("Shard " + shardInfo.shardId() + "'s only child shard " + childShard + + " has partial parent information in lease table."); + } + } + } + // Attempt create leases for child shards. for(ChildShard childShard : childShards) { final String leaseKey = ShardInfo.getLeaseKey(shardInfo, childShard.shardId()); if(leaseCoordinator.leaseRefresher().getLease(leaseKey) == null) { From d9f5557ff5157b44b449a037290f100c6a65766f Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Tue, 14 Jul 2020 01:58:01 -0700 Subject: [PATCH 139/159] Added unit test cases --- .../kinesis/lifecycle/ShutdownTaskTest.java | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) 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 792d566a..3405362a 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,6 +17,7 @@ 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.Mockito.doNothing; @@ -30,6 +31,7 @@ import java.util.Collections; import java.util.List; import java.util.UUID; +import com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -194,6 +196,52 @@ public class ShutdownTaskTest { verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class)); } + @Test + public final void testCallThrowsWhenParentInfoNotPresentInLease() 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, constructChildShard(), streamIdentifier, leaseCleanupManager); + + 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, 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, parentLease); + + // Make first attempt with partial parent info in lease table + TaskResult result = task.call(); + assertNotNull(result.getException()); + assertTrue(result.getException() instanceof InvalidStateException); + 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(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); + + // make next attempt with complete parent info in lease table + 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(leaseCoordinator).updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); + verify(leaseRefresher, times(1)).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class)); + + } + /** * Test method for {@link ShutdownTask#call()}. * This test is for the scenario that a ShutdownTask is created for detecting a false Shard End. @@ -269,4 +317,18 @@ public class ShutdownTaskTest { childShards.add(rightChild); return childShards; } + + private List constructChildShard() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(shardId); + parentShards.add("shardId-1"); + ChildShard leftChild = ChildShard.builder() + .shardId("shardId-2") + .parentShards(parentShards) + .hashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")) + .build(); + childShards.add(leftChild); + return childShards; + } } From 264e5bd4aa728e9f7354d431cfc118dc8b1686af Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 15 Jul 2020 10:08:50 -0700 Subject: [PATCH 140/159] Shutdown task to throw retryable exception intermittently to allow parent shard lease detection in merge cases. --- .../kinesis/lifecycle/ShutdownTask.java | 100 ++++++++++-------- .../kinesis/lifecycle/ShutdownTaskTest.java | 91 +++++++++++++--- 2 files changed, 128 insertions(+), 63 deletions(-) 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 d0d9c0c4..f72b31f8 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 @@ -25,11 +25,11 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.utils.CollectionUtils; -import software.amazon.awssdk.utils.Validate; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseCleanupManager; @@ -53,7 +53,6 @@ import software.amazon.kinesis.retrieval.RecordsPublisher; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.util.Set; -import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; @@ -100,6 +99,7 @@ public class ShutdownTask implements ConsumerTask { private final LeaseCleanupManager leaseCleanupManager; private static final Function leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo); + private int retryLeftForValidParentState = 10; /* * Invokes ShardRecordProcessor shutdown() API. @@ -121,26 +121,21 @@ public class ShutdownTask implements ConsumerTask { final long startTime = System.currentTimeMillis(); final Lease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(leaseKeyProvider.apply(shardInfo)); - final ShutdownReason localReason = attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(reason, currentShardLease); + final Runnable leaseLostAction = () -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()); - if (localReason == ShutdownReason.SHARD_END) { - final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, shardInfo); - if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { - boolean isSuccess = false; - try { - isSuccess = attemptShardEndCheckpointing(scope, startTime); - } finally { - // Check if either the shard end ddb persist is successful or - // if childshards is empty. When child shards is empty then either it is due to - // completed shard being reprocessed or we got RNF from service. - // For these cases enqueue the lease for deletion. - if (isSuccess || CollectionUtils.isNullOrEmpty(childShards)) { - leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); - } - } + if (reason == ShutdownReason.SHARD_END) { + try { + takeShardEndAction(currentShardLease, scope, startTime); + } catch (InvalidStateException e) { + // If InvalidStateException happens, it indicates we have a non recoverable error in short term. + // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry shutting down. + log.warn("Lease {}: Invalid state encountered while shutting down shardConsumer with SHARD_END reason. " + + "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. ", currentShardLease.leaseKey(), e); + dropLease(currentShardLease); + throwOnApplicationException(leaseLostAction, scope, startTime); } } else { - throwOnApplicationException(() -> shardRecordProcessor.leaseLost(LeaseLostInput.builder().build()), scope, startTime); + throwOnApplicationException(leaseLostAction, scope, startTime); } log.debug("Shutting down retrieval strategy for shard {}.", leaseKeyProvider.apply(shardInfo)); @@ -169,41 +164,47 @@ public class ShutdownTask implements ConsumerTask { return new TaskResult(exception); } - private ShutdownReason attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(ShutdownReason originalReason, Lease currentShardLease) - throws DependencyException, ProvisionedThroughputException { - ShutdownReason localReason = originalReason; - if (originalReason == ShutdownReason.SHARD_END) { - // Create new lease for the child shards if they don't exist. - // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. - // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. - // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. - // This scenario could happen when customer deletes the stream while leaving the KCL application running. - Validate.validState(currentShardLease != null, - "%s : Lease not owned by the current worker. Leaving ShardEnd handling to new owner.", - leaseKeyProvider.apply(shardInfo)); + // Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup. + private void takeShardEndAction(Lease currentShardLease, + MetricsScope scope, long startTime) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, + CustomerApplicationException { + // Create new lease for the child shards if they don't exist. + // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. + // This would happen when KinesisDataFetcher(for polling mode) or FanOutRecordsPublisher(for StoS mode) catches ResourceNotFound exception. + // In this case, KinesisDataFetcher and FanOutRecordsPublisher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. + // This scenario could happen when customer deletes the stream while leaving the KCL application running. + if (currentShardLease == null) { + throw new InvalidStateException(leaseKeyProvider.apply(shardInfo) + + " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner."); + } + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateLeaseWithChildShards(currentShardLease); + } + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, + shardInfo); + if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { + boolean isSuccess = false; try { - if (!CollectionUtils.isNullOrEmpty(childShards)) { - createLeasesForChildShardsIfNotExist(); - updateLeaseWithChildShards(currentShardLease); + isSuccess = attemptShardEndCheckpointing(scope, startTime); + } finally { + // Check if either the shard end ddb persist is successful or + // if childshards is empty. When child shards is empty then either it is due to + // completed shard being reprocessed or we got RNF from service. + // For these cases enqueue the lease for deletion. + if (isSuccess || CollectionUtils.isNullOrEmpty(childShards)) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); } - } catch (InvalidStateException e) { - // If InvalidStateException happens, it indicates we are missing childShard related information. - // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry getting - // childShard information in the processTask. - localReason = ShutdownReason.LEASE_LOST; - log.warn("Lease {}: Exception happened while shutting down shardConsumer with SHARD_END reason. " + - "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. Exception: ", currentShardLease.leaseKey(), e); - dropLease(currentShardLease); } } - return localReason; } private boolean attemptShardEndCheckpointing(MetricsScope scope, long startTime) throws DependencyException, ProvisionedThroughputException, InvalidStateException, CustomerApplicationException { final Lease leaseFromDdb = Optional.ofNullable(leaseCoordinator.leaseRefresher().getLease(leaseKeyProvider.apply(shardInfo))) - .orElseThrow(() -> new IllegalStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); + .orElseThrow(() -> new InvalidStateException("Lease for shard " + leaseKeyProvider.apply(shardInfo) + " does not exist.")); if (!leaseFromDdb.checkpoint().equals(ExtendedSequenceNumber.SHARD_END)) { // Call the shardRecordsProcessor to checkpoint with SHARD_END sequence number. // The shardEnded is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling shardEnded. @@ -252,8 +253,15 @@ public class ShutdownTask implements ConsumerTask { Objects.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(0))) == Objects .isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(1))); if (!isValidLeaseTableState) { - throw new InvalidStateException("Shard " + shardInfo.shardId() + "'s only child shard " + childShard - + " has partial parent information in lease table."); + if (--retryLeftForValidParentState >= 0) { + 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."); + } } } } 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 3405362a..dc5d9763 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 @@ -44,6 +44,7 @@ import software.amazon.kinesis.checkpoint.ShardRecordProcessorCheckpointer; import software.amazon.kinesis.common.InitialPositionInStream; import software.amazon.kinesis.common.InitialPositionInStreamExtended; import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.internal.BlockedOnParentShardException; import software.amazon.kinesis.exceptions.internal.KinesisClientLibIOException; import software.amazon.kinesis.leases.HierarchicalShardSyncer; import software.amazon.kinesis.leases.Lease; @@ -197,7 +198,7 @@ public class ShutdownTaskTest { } @Test - public final void testCallThrowsWhenParentInfoNotPresentInLease() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + public final void testCallThrowsUntilParentInfoNotPresentInLease() throws DependencyException, InvalidStateException, ProvisionedThroughputException { shardInfo = new ShardInfo("shardId-0", concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); task = new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, @@ -209,37 +210,93 @@ public class ShutdownTaskTest { 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, parentLease); + 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, parentLease); + when(leaseRefresher.getLease("shardId-1")) + .thenReturn(null, null, null, null, null, parentLease); - // Make first attempt with partial parent info in lease table - TaskResult result = task.call(); - assertNotNull(result.getException()); - assertTrue(result.getException() instanceof InvalidStateException); - 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(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); - verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); + // Make first 5 attempts with partial parent info in lease table + for (int i = 0; i < 5; i++) { + 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(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); + } // make next attempt with complete parent info in lease table - result = task.call(); + 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(leaseCoordinator).updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); + verify(leaseRefresher).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class)); verify(leaseRefresher, times(1)).createLeaseIfNotExists(Matchers.any(Lease.class)); verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); verify(leaseCleanupManager, times(1)).enqueueForDeletion(any(LeasePendingDeletion.class)); + } + @Test + public final void testCallTriggersLeaseLossWhenParentInfoNotPresentInLease() 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, constructChildShard(), streamIdentifier, leaseCleanupManager); + + when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2")); + Lease parentLease = LeaseHelper.createLease("shardId-1", "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-0")).thenReturn(heldLease); + when(leaseCoordinator.getCurrentlyHeldLease("shardId-1")) + .thenReturn(null, null, null, null, null, null, null, null, null, null, null); + when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); + when(leaseCoordinator.updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString())).thenReturn(true); + when(leaseRefresher.getLease("shardId-0")).thenReturn(heldLease); + // Return null lease first time to simulate partial parent lease info + when(leaseRefresher.getLease("shardId-1")) + .thenReturn(null, null, null, null, null, null, null, null, null, null, null); + + // Make first 10 attempts with partial parent info in lease table + for (int i = 0; i < 10; i++) { + 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(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); + } + + // make final attempt with incomplete parent info in lease table + 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(leaseCoordinator).dropLease(Matchers.any(Lease.class)); + verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); } /** From 7b704246e27944403f0b4aac05a6dbbf028d6111 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 15 Jul 2020 14:52:12 -0700 Subject: [PATCH 141/159] Fixing NPE in log --- .../java/software/amazon/kinesis/lifecycle/ShutdownTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f72b31f8..208498bc 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 @@ -130,7 +130,7 @@ public class ShutdownTask implements ConsumerTask { // If InvalidStateException happens, it indicates we have a non recoverable error in short term. // In this scenario, we should shutdown the shardConsumer with LEASE_LOST reason to allow other worker to take the lease and retry shutting down. log.warn("Lease {}: Invalid state encountered while shutting down shardConsumer with SHARD_END reason. " + - "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. ", currentShardLease.leaseKey(), e); + "Dropping the lease and shutting down shardConsumer using LEASE_LOST reason. ", leaseKeyProvider.apply(shardInfo), e); dropLease(currentShardLease); throwOnApplicationException(leaseLostAction, scope, startTime); } From 60b6066189a3781442e0ce5fbaf5911e7d8e32df Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 16 Jul 2020 12:14:53 -0700 Subject: [PATCH 142/159] Breaking loop on ShutdownException using 10-faced dice --- .../kinesis/lifecycle/ShutdownTask.java | 15 ++++++-- .../kinesis/lifecycle/ShutdownTaskTest.java | 36 ++++++++++++++----- 2 files changed, 40 insertions(+), 11 deletions(-) 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 208498bc..6b4d1839 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 @@ -52,6 +52,7 @@ import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.RecordsPublisher; 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; @@ -65,6 +66,8 @@ 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"; + @VisibleForTesting + static final int RETRY_RANDOM_MAX_RANGE = 10; @NonNull private final ShardInfo shardInfo; @@ -99,7 +102,6 @@ public class ShutdownTask implements ConsumerTask { private final LeaseCleanupManager leaseCleanupManager; private static final Function leaseKeyProvider = shardInfo -> ShardInfo.getLeaseKey(shardInfo); - private int retryLeftForValidParentState = 10; /* * Invokes ShardRecordProcessor shutdown() API. @@ -253,7 +255,7 @@ public class ShutdownTask implements ConsumerTask { Objects.isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(0))) == Objects .isNull(leaseCoordinator.leaseRefresher().getLease(parentLeaseKeys.get(1))); if (!isValidLeaseTableState) { - if (--retryLeftForValidParentState >= 0) { + 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."); @@ -276,6 +278,15 @@ public class ShutdownTask implements ConsumerTask { } } + /** + * Returns true for 1 in N probability. + */ + @VisibleForTesting + boolean isOneInNProbability(int n) { + Random r = new Random(); + return 1 == r.nextInt((n - 1) + 1) + 1; + } + private void updateLeaseWithChildShards(Lease currentLease) throws DependencyException, InvalidStateException, ProvisionedThroughputException { Set childShardIds = childShards.stream().map(ChildShard::shardId).collect(Collectors.toSet()); 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 dc5d9763..688bd199 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/lifecycle/ShutdownTaskTest.java @@ -20,11 +20,14 @@ 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; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static software.amazon.kinesis.lifecycle.ShutdownTask.RETRY_RANDOM_MAX_RANGE; import java.util.ArrayList; import java.util.Collections; @@ -201,11 +204,6 @@ public class ShutdownTaskTest { public final void testCallThrowsUntilParentInfoNotPresentInLease() 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, constructChildShard(), streamIdentifier, leaseCleanupManager); - 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()); @@ -221,6 +219,11 @@ public class ShutdownTaskTest { // 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); @@ -232,11 +235,17 @@ public class ShutdownTaskTest { 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(); @@ -244,6 +253,7 @@ public class ShutdownTaskTest { 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)); } @@ -252,10 +262,6 @@ public class ShutdownTaskTest { public final void testCallTriggersLeaseLossWhenParentInfoNotPresentInLease() 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, constructChildShard(), streamIdentifier, leaseCleanupManager); when(recordProcessorCheckpointer.lastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); Lease heldLease = LeaseHelper.createLease("shardId-0", "leaseOwner", ImmutableList.of("parent1", "parent2")); @@ -272,6 +278,11 @@ public class ShutdownTaskTest { // Make first 10 attempts with partial parent info in lease table for (int i = 0; i < 10; i++) { + ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, + SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager)); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); assertTrue(result.getException() instanceof BlockedOnParentShardException); @@ -283,11 +294,17 @@ public class ShutdownTaskTest { verify(leaseCoordinator, never()) .updateLease(Matchers.any(Lease.class), Matchers.any(UUID.class), Matchers.anyString(), Matchers.anyString()); verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); verify(leaseCoordinator, never()).dropLease(Matchers.any(Lease.class)); verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); } // make final attempt with incomplete parent info in lease table + ShutdownTask task = spy(new ShutdownTask(shardInfo, shardDetector, shardRecordProcessor, recordProcessorCheckpointer, + SHARD_END_SHUTDOWN_REASON, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseCoordinator, TASK_BACKOFF_TIME_MILLIS, recordsPublisher, + hierarchicalShardSyncer, NULL_METRICS_FACTORY, constructChildShard(), streamIdentifier, leaseCleanupManager)); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(true); TaskResult result = task.call(); assertNull(result.getException()); verify(recordsPublisher).shutdown(); @@ -295,6 +312,7 @@ public class ShutdownTaskTest { verify(shardRecordProcessor).leaseLost(LeaseLostInput.builder().build()); verify(leaseRefresher, never()).updateLeaseWithMetaInfo(Matchers.any(Lease.class), Matchers.any(UpdateField.class)); verify(leaseRefresher, never()).createLeaseIfNotExists(Matchers.any(Lease.class)); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); verify(leaseCoordinator).dropLease(Matchers.any(Lease.class)); verify(leaseCleanupManager, never()).enqueueForDeletion(any(LeasePendingDeletion.class)); } From 4c899b2930d3f22e2c591a284b6e60d6688e5678 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Thu, 16 Jul 2020 20:34:48 -0700 Subject: [PATCH 143/159] Log details of ResourceNotFoundException while cleaning up leases --- .../amazon/kinesis/leases/LeaseCleanupManager.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 119402cb..4b11b627 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 @@ -210,7 +210,7 @@ public class LeaseCleanupManager { } } catch (ResourceNotFoundException e) { wasResourceNotFound = true; - cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease); + cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease, e); } return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease, wereChildShardsPresent, @@ -242,8 +242,8 @@ 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) throws DependencyException, ProvisionedThroughputException, InvalidStateException { - log.info("Deleting lease {} as it is not present in the stream.", lease); + 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; } @@ -333,7 +333,7 @@ public class LeaseCleanupManager { garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease(); if (leaseCleanupResult.leaseCleanedUp()) { - log.debug("Successfully cleaned up lease {} for {}", leaseKey, streamIdentifier); + log.info("Successfully cleaned up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult); deletionSucceeded = true; } else { log.warn("Unable to clean up lease {} for {} due to {}", leaseKey, streamIdentifier, leaseCleanupResult); From c68ab705bd536b0fbaa1ad5872e59b41f727bbdb Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Fri, 17 Jul 2020 14:48:25 -0700 Subject: [PATCH 144/159] Fix NonGreedyTake integration test and add extra integration test (#80) * Fix NonGreedyTake integration test and add extra integration test for very old leases * Fix spacing in comment --- .../leases/dynamodb/DynamoDBLeaseTaker.java | 16 ++++++++++-- .../DynamoDBLeaseTakerIntegrationTest.java | 26 +++++++++++++++---- 2 files changed, 35 insertions(+), 7 deletions(-) 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 3ec2de22..4249f32b 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTaker.java @@ -48,7 +48,7 @@ import software.amazon.kinesis.metrics.MetricsUtil; public class DynamoDBLeaseTaker implements LeaseTaker { private static final int TAKE_RETRIES = 3; private static final int SCAN_RETRIES = 1; - private static final int VERY_OLD_LEASE_DURATION_NANOS_MULTIPLIER = 3; + private long veryOldLeaseDurationNanosMultiplier = 3; // See note on TAKE_LEASES_DIMENSION(Callable) for why we have this callable. private static final Callable SYSTEM_CLOCK_CALLABLE = System::nanoTime; @@ -96,6 +96,18 @@ public class DynamoDBLeaseTaker implements LeaseTaker { return this; } + /** + * Overrides the default very old lease duration nanos multiplier to increase the threshold for taking very old leases. + * Setting this to a higher value than 3 will increase the threshold for very old lease taking. + * + * @param veryOldLeaseDurationNanosMultipler Very old lease duration multiplier for adjusting very old lease taking. + * @return LeaseTaker + */ + public DynamoDBLeaseTaker withVeryOldLeaseDurationNanosMultipler(long veryOldLeaseDurationNanosMultipler) { + this.veryOldLeaseDurationNanosMultiplier = veryOldLeaseDurationNanosMultipler; + return this; + } + /** * Max leases to steal from a more loaded Worker at one time (for load balancing). * Setting this to a higher number can allow for faster load convergence (e.g. during deployments, cold starts), @@ -382,7 +394,7 @@ public class DynamoDBLeaseTaker implements LeaseTaker { // later) but obeying the maximum limit per worker. veryOldLeases = allLeases.values().stream() .filter(lease -> System.nanoTime() - lease.lastCounterIncrementNanos() - > VERY_OLD_LEASE_DURATION_NANOS_MULTIPLIER * leaseDurationNanos) + > veryOldLeaseDurationNanosMultiplier * leaseDurationNanos) .collect(Collectors.toList()); if (!veryOldLeases.isEmpty()) { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java index 2e3d65a4..972d3951 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseTakerIntegrationTest.java @@ -16,22 +16,17 @@ package software.amazon.kinesis.leases.dynamodb; import java.util.Collection; import java.util.Map; - import org.junit.Assert; import org.junit.Before; import org.junit.Test; - import org.junit.runner.RunWith; import org.mockito.runners.MockitoJUnitRunner; import software.amazon.kinesis.leases.Lease; import software.amazon.kinesis.leases.LeaseIntegrationTest; import software.amazon.kinesis.leases.exceptions.LeasingException; import software.amazon.kinesis.metrics.NullMetricsFactory; - import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; @RunWith(MockitoJUnitRunner.class) public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest { @@ -105,9 +100,30 @@ public class DynamoDBLeaseTakerIntegrationTest extends LeaseIntegrationTest { builder.withLease("4", "bar").build(); + // setting multiplier to unusually high number to avoid very old lease taking + taker.withVeryOldLeaseDurationNanosMultipler(5000000000L); builder.takeMutateAssert(taker, 2); } + /** + * Verify that we take all very old leases by setting up an environment where there are 4 leases and 2 workers, + * only one of which holds a lease. This leaves 3 free leases. LeaseTaker should take all 3 leases since they + * are denoted as very old. + */ + @Test + public void testVeryOldLeaseTaker() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + + for (int i = 0; i < 3; i++) { + builder.withLease(Integer.toString(i), null); + } + + builder.withLease("4", "bar").build(); + + // setting multiplier to unusually high number to avoid very old lease taking + builder.takeMutateAssert(taker, 3); + } + /** * Verify that when getAllLeases() is called, DynamoDBLeaseTaker * - does not call listLeases() From ff703459e17777ea727c0884758849b12c10a2ac Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Fri, 17 Jul 2020 18:32:50 -0400 Subject: [PATCH 145/159] Exposing interface for getting child shards (#81) * Exposing interface for getting child shards * javadocs * more java docs * Addressing comments * Tuning random max range Co-authored-by: Joshua Kim --- .../kinesis/leases/KinesisShardDetector.java | 27 ++++++++++++ .../kinesis/leases/LeaseCleanupManager.java | 42 +------------------ .../amazon/kinesis/leases/ShardDetector.java | 24 ++++++++++- .../DynamoDBLeaseManagementFactory.java | 4 +- .../exceptions/LeasePendingDeletion.java | 18 ++++++++ .../kinesis/lifecycle/ShutdownTask.java | 4 +- .../leases/KinesisShardDetectorTest.java | 18 ++++---- .../leases/LeaseCleanupManagerTest.java | 42 ++++++------------- 8 files changed, 96 insertions(+), 83 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 f5ef482e..96a0de6a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/KinesisShardDetector.java @@ -35,6 +35,11 @@ import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.ChildShard; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorResponse; import software.amazon.awssdk.services.kinesis.model.KinesisException; import software.amazon.awssdk.services.kinesis.model.LimitExceededException; import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; @@ -43,6 +48,7 @@ import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.Shard; import software.amazon.awssdk.services.kinesis.model.ShardFilter; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; @@ -262,4 +268,25 @@ public class KinesisShardDetector implements ShardDetector { ExecutionException, TimeoutException, InterruptedException { return FutureUtils.resolveOrCancelFuture(kinesisClient.listShards(request), kinesisRequestTimeout); } + + @Override + public List getChildShards(final String shardId) throws InterruptedException, ExecutionException, TimeoutException { + final GetShardIteratorRequest getShardIteratorRequest = KinesisRequestsBuilder.getShardIteratorRequestBuilder() + .streamName(streamIdentifier.streamName()) + .shardIteratorType(ShardIteratorType.LATEST) + .shardId(shardId) + .build(); + + final GetShardIteratorResponse getShardIteratorResponse = + FutureUtils.resolveOrCancelFuture(kinesisClient.getShardIterator(getShardIteratorRequest), kinesisRequestTimeout); + + final GetRecordsRequest getRecordsRequest = KinesisRequestsBuilder.getRecordsRequestBuilder() + .shardIterator(getShardIteratorResponse.shardIterator()) + .build(); + + final GetRecordsResponse getRecordsResponse = + FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(getRecordsRequest), kinesisRequestTimeout); + + return getRecordsResponse.childShards(); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseCleanupManager.java index 4b11b627..de734646 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 @@ -24,16 +24,8 @@ import lombok.RequiredArgsConstructor; import lombok.Value; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; -import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; -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.ResourceNotFoundException; -import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; import software.amazon.awssdk.utils.CollectionUtils; -import software.amazon.kinesis.common.FutureUtils; -import software.amazon.kinesis.common.KinesisRequestsBuilder; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.exceptions.DependencyException; import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; @@ -43,8 +35,6 @@ import software.amazon.kinesis.metrics.MetricsFactory; import software.amazon.kinesis.retrieval.AWSExceptionManager; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.time.Duration; -import java.util.Collections; import java.util.HashSet; import java.util.Objects; import java.util.Optional; @@ -69,12 +59,8 @@ public class LeaseCleanupManager { @NonNull private final LeaseCoordinator leaseCoordinator; @NonNull - private final KinesisAsyncClient kinesisClient; - @NonNull private final MetricsFactory metricsFactory; @NonNull - private final Duration maxFutureWait; - @NonNull private final ScheduledExecutorService deletionThreadPool; private final boolean cleanupLeasesUponShardCompletion; private final long leaseCleanupIntervalMillis; @@ -85,7 +71,6 @@ public class LeaseCleanupManager { private final Queue deletionQueue = new ConcurrentLinkedQueue<>(); - private static final int MAX_RECORDS = 1; private static final long INITIAL_DELAY = 0L; @Getter @@ -170,7 +155,7 @@ public class LeaseCleanupManager { Set childShardKeys = leaseFromDDB.childShardIds(); if (CollectionUtils.isNullOrEmpty(childShardKeys)) { try { - childShardKeys = getChildShardsFromService(shardInfo, streamIdentifier); + childShardKeys = leasePendingDeletion.getChildShardsFromService(); if (CollectionUtils.isNullOrEmpty(childShardKeys)) { log.error( @@ -203,7 +188,7 @@ public class LeaseCleanupManager { if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) { try { wereChildShardsPresent = !CollectionUtils - .isNullOrEmpty(getChildShardsFromService(shardInfo, streamIdentifier)); + .isNullOrEmpty(leasePendingDeletion.getChildShardsFromService()); } catch (ExecutionException e) { throw exceptionManager.apply(e.getCause()); } @@ -217,29 +202,6 @@ public class LeaseCleanupManager { wasResourceNotFound); } - private Set getChildShardsFromService(ShardInfo shardInfo, StreamIdentifier streamIdentifier) - throws InterruptedException, ExecutionException, TimeoutException { - final GetShardIteratorRequest getShardIteratorRequest = KinesisRequestsBuilder.getShardIteratorRequestBuilder() - .streamName(streamIdentifier.streamName()) - .shardIteratorType(ShardIteratorType.LATEST) - .shardId(shardInfo.shardId()) - .build(); - - final GetShardIteratorResponse getShardIteratorResponse = - FutureUtils.resolveOrCancelFuture(kinesisClient.getShardIterator(getShardIteratorRequest), maxFutureWait); - - final GetRecordsRequest getRecordsRequest = KinesisRequestsBuilder.getRecordsRequestBuilder() - .shardIterator(getShardIteratorResponse.shardIterator()) - .limit(MAX_RECORDS) - .build(); - - final GetRecordsResponse getRecordsResponse = - FutureUtils.resolveOrCancelFuture(kinesisClient.getRecords(getRecordsRequest), maxFutureWait); - - return getRecordsResponse.childShards().stream().map(c -> c.shardId()).collect(Collectors.toSet()); - } - - // A lease that ended with SHARD_END from ResourceNotFoundException is safe to delete if it no longer exists in the // stream (known explicitly from ResourceNotFound being thrown when processing this shard), private boolean cleanupLeaseForGarbageShard(Lease lease, Throwable e) throws DependencyException, ProvisionedThroughputException, InvalidStateException { 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 9eb2d17b..62b93855 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardDetector.java @@ -16,6 +16,10 @@ package software.amazon.kinesis.leases; import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +import software.amazon.awssdk.services.kinesis.model.ChildShard; import software.amazon.awssdk.services.kinesis.model.ListShardsRequest; import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.Shard; @@ -48,7 +52,9 @@ public interface ShardDetector { * @param ShardFilter * @return Shards */ - List listShardsWithFilter(ShardFilter shardFilter); + default List listShardsWithFilter(ShardFilter shardFilter) { + throw new UnsupportedOperationException("listShardsWithFilter not available."); + } /** * Gets stream identifier. @@ -65,5 +71,19 @@ public interface ShardDetector { * @param request list shards request * @return ListShardsResponse which contains list shards response */ - ListShardsResponse getListShardsResponse(ListShardsRequest request) throws Exception; + default ListShardsResponse getListShardsResponse(ListShardsRequest request) throws Exception { + throw new UnsupportedOperationException("getListShardsResponse not available."); + } + + /** + * Gets the children shards of a shard. + * @param shardId + * @return + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + default List getChildShards(String shardId) throws InterruptedException, ExecutionException, TimeoutException { + throw new UnsupportedOperationException("getChildShards not available."); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java index 7d374de5..5102bc5e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseManagementFactory.java @@ -551,8 +551,8 @@ public class DynamoDBLeaseManagementFactory implements LeaseManagementFactory { */ @Override public LeaseCleanupManager createLeaseCleanupManager(MetricsFactory metricsFactory) { - return new LeaseCleanupManager(createLeaseCoordinator(metricsFactory), kinesisClient, - metricsFactory, dynamoDbRequestTimeout, Executors.newSingleThreadScheduledExecutor(), + return new LeaseCleanupManager(createLeaseCoordinator(metricsFactory), + metricsFactory, Executors.newSingleThreadScheduledExecutor(), cleanupLeasesUponShardCompletion, leaseCleanupConfig.leaseCleanupIntervalMillis(), leaseCleanupConfig.completedLeaseCleanupIntervalMillis(), leaseCleanupConfig.garbageLeaseCleanupIntervalMillis()); diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java index b840eb09..2d3d0c2f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/exceptions/LeasePendingDeletion.java @@ -20,8 +20,14 @@ import lombok.Value; import lombok.experimental.Accessors; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.ShardDetector; import software.amazon.kinesis.leases.ShardInfo; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + /** * Helper class for cleaning up leases. */ @@ -32,4 +38,16 @@ public class LeasePendingDeletion { private final StreamIdentifier streamIdentifier; private final Lease lease; private final ShardInfo shardInfo; + private final ShardDetector shardDetector; + + /** + * Discovers the child shards for this lease. + * @return + * @throws InterruptedException + * @throws ExecutionException + * @throws TimeoutException + */ + public Set getChildShardsFromService() throws InterruptedException, ExecutionException, TimeoutException { + return shardDetector.getChildShards(shardInfo.shardId()).stream().map(c -> c.shardId()).collect(Collectors.toSet()); + } } diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java index 6b4d1839..c2c5c790 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/lifecycle/ShutdownTask.java @@ -67,7 +67,7 @@ public class ShutdownTask implements ConsumerTask { private static final String SHUTDOWN_TASK_OPERATION = "ShutdownTask"; private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown"; @VisibleForTesting - static final int RETRY_RANDOM_MAX_RANGE = 10; + static final int RETRY_RANDOM_MAX_RANGE = 30; @NonNull private final ShardInfo shardInfo; @@ -185,7 +185,7 @@ public class ShutdownTask implements ConsumerTask { updateLeaseWithChildShards(currentShardLease); } final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(streamIdentifier, currentShardLease, - shardInfo); + shardInfo, shardDetector); if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { boolean isSuccess = false; try { diff --git a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java index 68bb7d97..d0870d51 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/KinesisShardDetectorTest.java @@ -15,14 +15,6 @@ package software.amazon.kinesis.leases; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -38,6 +30,16 @@ import software.amazon.awssdk.services.kinesis.model.ListShardsResponse; import software.amazon.awssdk.services.kinesis.model.ResourceInUseException; import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; import software.amazon.awssdk.services.kinesis.model.Shard; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.isA; import static org.hamcrest.CoreMatchers.nullValue; 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 d02ced04..e9d237f9 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 @@ -15,18 +15,12 @@ package software.amazon.kinesis.leases; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import software.amazon.awssdk.services.kinesis.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.ResourceNotFoundException; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.leases.exceptions.LeasePendingDeletion; @@ -39,7 +33,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.UUID; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; @@ -73,7 +66,7 @@ public class LeaseCleanupManagerTest { @Mock private LeaseCoordinator leaseCoordinator; @Mock - private KinesisAsyncClient kinesis; + private ShardDetector shardDetector; @Mock private ScheduledExecutorService deletionThreadPool; @@ -82,9 +75,9 @@ public class LeaseCleanupManagerTest { shardInfo = new ShardInfo(shardId, concurrencyToken, Collections.emptySet(), ExtendedSequenceNumber.LATEST); streamIdentifier = StreamIdentifier.singleStreamInstance("streamName"); - leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, - NULL_METRICS_FACTORY, maxFutureWait, deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, - completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + garbageLeaseCleanupIntervalMillis); when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); when(leaseCoordinator.updateLease(any(Lease.class), any(UUID.class), any(String.class), any(String.class))).thenReturn(true); @@ -124,8 +117,8 @@ public class LeaseCleanupManagerTest { ExtendedSequenceNumber.LATEST); cleanupLeasesOfCompletedShards = false; - leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, NULL_METRICS_FACTORY, maxFutureWait, - deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShardsForSplit(), ExtendedSequenceNumber.LATEST, 0); @@ -206,8 +199,8 @@ public class LeaseCleanupManagerTest { cleanupLeasesOfCompletedShards = false; - leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, kinesis, NULL_METRICS_FACTORY, maxFutureWait, - deletionThreadPool, cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, + leaseCleanupManager = new LeaseCleanupManager(leaseCoordinator, NULL_METRICS_FACTORY, deletionThreadPool, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis); testLeaseDeletedWhenShardDoesNotExist(heldLease); @@ -216,12 +209,13 @@ public class LeaseCleanupManagerTest { public void testLeaseDeletedWhenShardDoesNotExist(Lease heldLease) throws Exception { when(leaseCoordinator.leaseRefresher()).thenReturn(leaseRefresher); when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.shardId())).thenReturn(heldLease); - when(kinesis.getShardIterator(any(GetShardIteratorRequest.class))).thenThrow(ResourceNotFoundException.class); + when(shardDetector.getChildShards(any(String.class))).thenThrow(ResourceNotFoundException.class); when(leaseRefresher.getLease(heldLease.leaseKey())).thenReturn(heldLease); - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo)); + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, heldLease, shardInfo, shardDetector)); leaseCleanupManager.cleanupLeases(); + verify(shardDetector, times(1)).getChildShards(shardInfo.shardId()); verify(leaseRefresher, times(1)).deleteLease(heldLease); } @@ -256,20 +250,10 @@ public class LeaseCleanupManagerTest { } } - GetShardIteratorResponse getShardIteratorResponse = GetShardIteratorResponse.builder() - .shardIterator("123") - .build(); - when(kinesis.getShardIterator(any(GetShardIteratorRequest.class))).thenReturn(CompletableFuture.completedFuture(getShardIteratorResponse)); - - GetRecordsResponse getRecordsResponse = GetRecordsResponse.builder() - .records(Collections.emptyList()) - .childShards(childShards) - .build(); - when(kinesis.getRecords(any(GetRecordsRequest.class))).thenReturn(CompletableFuture.completedFuture(getRecordsResponse)); - - leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, lease, shardInfo)); + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(streamIdentifier, lease, shardInfo, shardDetector)); leaseCleanupManager.cleanupLeases(); + verify(shardDetector, times(1)).getChildShards(shardInfo.shardId()); verify(leaseRefresher, times(expectedDeletedLeases)).deleteLease(any(Lease.class)); } From 8aec062e64f2a270e9f7de66d1c0e87780c9eae9 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 22 Jul 2020 10:36:22 -0700 Subject: [PATCH 146/159] Avoiding ShardSync Task sleep when we skip the shard sync due to no shards found --- .../amazon/kinesis/leases/HierarchicalShardSyncer.java | 10 ++++++---- .../software/amazon/kinesis/leases/ShardSyncTask.java | 4 ++-- 2 files changed, 8 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 b3cfdb56..4f677524 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/HierarchicalShardSyncer.java @@ -97,6 +97,7 @@ public class HierarchicalShardSyncer { /** * Check and create leases for any new shards (e.g. following a reshard operation). Sync leases with Kinesis shards * (e.g. at startup, or when we reach end of a shard). + * Return true, if shard sync was performed. Return false, if shard sync is skipped. * * @param shardDetector * @param leaseRefresher @@ -109,18 +110,18 @@ public class HierarchicalShardSyncer { * @throws KinesisClientLibIOException */ // CHECKSTYLE:OFF CyclomaticComplexity - public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, + public synchronized boolean checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, final MetricsScope scope, final boolean ignoreUnexpectedChildShards, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException, InterruptedException { final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(shardDetector, initialPosition) : getShardList(shardDetector); - checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, ignoreUnexpectedChildShards, scope, + return checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, latestShards, ignoreUnexpectedChildShards, scope, isLeaseTableEmpty); } //Provide a pre-collcted list of shards to avoid calling ListShards API - public synchronized void checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, + public synchronized boolean checkAndCreateLeaseForNewShards(@NonNull final ShardDetector shardDetector, final LeaseRefresher leaseRefresher, final InitialPositionInStreamExtended initialPosition, List latestShards, final boolean ignoreUnexpectedChildShards, final MetricsScope scope, final boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { @@ -131,7 +132,7 @@ public class HierarchicalShardSyncer { log.debug("{} - Num shards: {}", streamIdentifier, latestShards.size()); } else { log.warn("Skipping shard sync for {} as no shards found from service.", streamIdentifier); - return; + return false; } final Map shardIdToShardMap = constructShardIdToShardMap(latestShards); @@ -161,6 +162,7 @@ public class HierarchicalShardSyncer { } final List trackedLeases = new ArrayList<>(currentLeases); trackedLeases.addAll(newLeasesToCreate); + return true; } /** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls 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 820d4528..dd576114 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTask.java @@ -67,11 +67,11 @@ public class ShardSyncTask implements ConsumerTask { boolean shardSyncSuccess = true; try { - hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, + boolean didPerformShardSync = hierarchicalShardSyncer.checkAndCreateLeaseForNewShards(shardDetector, leaseRefresher, initialPosition, scope, ignoreUnexpectedChildShards, leaseRefresher.isLeaseTableEmpty()); - if (shardSyncTaskIdleTimeMillis > 0) { + if (didPerformShardSync && shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); } } catch (Exception e) { From 75fe6060f6dd001052a8826cf9d7979001671152 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Wed, 22 Jul 2020 11:37:25 -0700 Subject: [PATCH 147/159] Adding logging around the shard sync executor queuesize --- .../amazon/kinesis/coordinator/ExecutorStateEvent.java | 9 ++++++--- .../amazon/kinesis/leases/ShardSyncTaskManager.java | 5 ++++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java index 3333cc42..33c83a5c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/ExecutorStateEvent.java @@ -28,7 +28,7 @@ import java.util.concurrent.ThreadPoolExecutor; @ToString(exclude = "isThreadPoolExecutor") @Slf4j @KinesisClientInternalApi -class ExecutorStateEvent implements DiagnosticEvent { +public class ExecutorStateEvent implements DiagnosticEvent { private static final String MESSAGE = "Current thread pool executor state: "; private boolean isThreadPoolExecutor; @@ -41,6 +41,11 @@ class ExecutorStateEvent implements DiagnosticEvent { private int maximumPoolSize; ExecutorStateEvent(ExecutorService executor, LeaseCoordinator leaseCoordinator) { + this(executor); + this.leasesOwned = leaseCoordinator.getAssignments().size(); + } + + public ExecutorStateEvent(ExecutorService executor) { if (executor instanceof ThreadPoolExecutor) { this.isThreadPoolExecutor = true; @@ -52,8 +57,6 @@ class ExecutorStateEvent implements DiagnosticEvent { this.largestPoolSize = ex.getLargestPoolSize(); this.maximumPoolSize = ex.getMaximumPoolSize(); } - - this.leasesOwned = leaseCoordinator.getAssignments().size(); } @Override diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java index 6a1ceff4..e03046a0 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/ShardSyncTaskManager.java @@ -17,7 +17,6 @@ package software.amazon.kinesis.leases; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; @@ -27,6 +26,7 @@ import lombok.Data; import lombok.NonNull; import lombok.experimental.Accessors; import lombok.extern.slf4j.Slf4j; +import software.amazon.kinesis.coordinator.ExecutorStateEvent; import software.amazon.kinesis.lifecycle.ConsumerTask; import software.amazon.kinesis.lifecycle.TaskResult; import software.amazon.kinesis.metrics.MetricsFactory; @@ -186,6 +186,9 @@ public class ShardSyncTaskManager { metricsFactory); future = CompletableFuture.supplyAsync(() -> currentTask.call(), executorService) .whenComplete((taskResult, exception) -> handlePendingShardSyncs(exception, taskResult)); + + log.info(new ExecutorStateEvent(executorService).message()); + submittedNewTask = true; if (log.isDebugEnabled()) { log.debug("Submitted new {} task.", currentTask.taskType()); From 44f2cb7d5becfdbaabaddd331ecf7ef6dc2c0b3e Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 21 Jul 2020 06:09:27 -0400 Subject: [PATCH 148/159] Pom change to 2.10.65 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f21b230e..4e8da722 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.11.8-SNAPSHOT + 2.10.65-SNAPSHOT From bc5dae270f249ec036bbbae1ccf824a15f7f1c94 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 17:58:39 -0400 Subject: [PATCH 149/159] Fixing bug where isRunning is not set --- .../java/software/amazon/kinesis/leases/LeaseCleanupManager.java | 1 + 1 file changed, 1 insertion(+) 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 de734646..4e43c638 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 @@ -82,6 +82,7 @@ public class LeaseCleanupManager { */ public void start() { log.debug("Starting lease cleanup thread."); + isRunning = true; completedLeaseStopwatch.start(); garbageLeaseStopwatch.start(); From 5470bc74880478b1097e50f0f39fdd129c1fb16a Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 17:59:29 -0400 Subject: [PATCH 150/159] Revert "Pom change to 2.10.65" This reverts commit 44f2cb7d5becfdbaabaddd331ecf7ef6dc2c0b3e. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4e8da722..f21b230e 100644 --- a/pom.xml +++ b/pom.xml @@ -33,7 +33,7 @@ - 2.10.65-SNAPSHOT + 2.11.8-SNAPSHOT From 5a0a6c600e80d39d81764be4fb7579faf45f8a6a Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 18:08:35 -0400 Subject: [PATCH 151/159] Adding safety check around stopwatches --- .../amazon/kinesis/leases/LeaseCleanupManager.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 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 4e43c638..0feb25e6 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 @@ -82,12 +82,15 @@ public class LeaseCleanupManager { */ public void start() { log.debug("Starting lease cleanup thread."); - isRunning = true; - completedLeaseStopwatch.start(); - garbageLeaseStopwatch.start(); + + if (!isRunning) { + completedLeaseStopwatch.start(); + garbageLeaseStopwatch.start(); + } deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, TimeUnit.MILLISECONDS); + isRunning = true; } /** From 9c8b89e0c6a96390d507c59be23a1de688658a54 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 18:14:11 -0400 Subject: [PATCH 152/159] Moving to after watches and executor initialized --- .../amazon/kinesis/leases/LeaseCleanupManager.java | 8 +++----- 1 file changed, 3 insertions(+), 5 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 0feb25e6..1d73c852 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 @@ -83,10 +83,8 @@ public class LeaseCleanupManager { public void start() { log.debug("Starting lease cleanup thread."); - if (!isRunning) { - completedLeaseStopwatch.start(); - garbageLeaseStopwatch.start(); - } + completedLeaseStopwatch.start(); + garbageLeaseStopwatch.start(); deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, TimeUnit.MILLISECONDS); @@ -138,7 +136,7 @@ public class LeaseCleanupManager { } public LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion, - boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException, + boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException, InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { final Lease lease = leasePendingDeletion.lease(); final ShardInfo shardInfo = leasePendingDeletion.shardInfo(); From cadbc0503b8eec5f974c2bd0da80504b7ea504cc Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 18:15:57 -0400 Subject: [PATCH 153/159] reverting whitespace --- .../software/amazon/kinesis/leases/LeaseCleanupManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1d73c852..b725696c 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 @@ -136,7 +136,7 @@ public class LeaseCleanupManager { } public LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion, - boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException, + boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) throws TimeoutException, InterruptedException, DependencyException, ProvisionedThroughputException, InvalidStateException { final Lease lease = leasePendingDeletion.lease(); final ShardInfo shardInfo = leasePendingDeletion.shardInfo(); From 15c45d2029af722b3e724b3e99f95b2930f8690e Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 19:17:27 -0400 Subject: [PATCH 154/159] Adding a unit test --- .../kinesis/leases/LeaseCleanupManager.java | 18 ++++++++++-------- .../leases/LeaseCleanupManagerTest.java | 11 +++++++++++ 2 files changed, 21 insertions(+), 8 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 b725696c..0fb9ccd6 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 @@ -81,14 +81,16 @@ public class LeaseCleanupManager { * {@link LeaseCleanupManager#leaseCleanupIntervalMillis} */ public void start() { - log.debug("Starting lease cleanup thread."); - - completedLeaseStopwatch.start(); - garbageLeaseStopwatch.start(); - - deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, - TimeUnit.MILLISECONDS); - isRunning = true; + if (!isRunning) { + log.debug("Starting lease cleanup thread."); + completedLeaseStopwatch.reset().start(); + garbageLeaseStopwatch.reset().start(); + deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, + TimeUnit.MILLISECONDS); + isRunning = true; + } else { + log.debug("Lease cleanup thread already running, no need to start."); + } } /** 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 e9d237f9..02c71b03 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 @@ -15,6 +15,7 @@ package software.amazon.kinesis.leases; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -83,6 +84,16 @@ public class LeaseCleanupManagerTest { when(leaseCoordinator.updateLease(any(Lease.class), any(UUID.class), any(String.class), any(String.class))).thenReturn(true); } + /** + * Tests subsequent calls to start {@link LeaseCleanupManager}. + */ + @Test + public final void testSubsequentStarts() { + leaseCleanupManager.start(); + Assert.assertTrue(leaseCleanupManager.isRunning()); + leaseCleanupManager.start(); + } + /** * Tests that when both child shard leases are present, we are able to delete the parent shard for the completed * shard case. From bdf019748f9c41f65b634c9dc30e071669eca1ec Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 27 Jul 2020 16:26:51 -0700 Subject: [PATCH 155/159] Add conditional check while updating the lease table meta info --- .../amazon/kinesis/leases/LeaseSerializer.java | 7 +++++++ .../leases/dynamodb/DynamoDBLeaseRefresher.java | 5 +++++ .../leases/dynamodb/DynamoDBLeaseSerializer.java | 10 ++++++++++ 3 files changed, 22 insertions(+) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java index 5dbf6366..09d2280e 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java @@ -81,6 +81,13 @@ public interface LeaseSerializer { */ Map getDynamoNonexistantExpectation(); + /** + * @return the attribute value map asserting that a lease does exist. + */ + default Map getDynamoExistantExpectation() { + throw new UnsupportedOperationException(); + } + /** * @param lease * @return the attribute value map that increments a lease counter 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 df5746a2..98f75144 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 @@ -681,8 +681,10 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { throws DependencyException, InvalidStateException, ProvisionedThroughputException { log.debug("Updating lease without expectation {}", lease); final AWSExceptionManager exceptionManager = createExceptionManager(); + exceptionManager.add(ConditionalCheckFailedException.class, t -> t); Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); UpdateItemRequest request = UpdateItemRequest.builder().tableName(table).key(serializer.getDynamoHashKey(lease)) + .expected(serializer.getDynamoExistantExpectation()) .attributeUpdates(updates).build(); try { try { @@ -692,6 +694,9 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { } catch (InterruptedException e) { throw new DependencyException(e); } + } catch (ConditionalCheckFailedException e) { + log.warn("Lease update failed for lease with key {} because the lease did not exist at the time of the update", + lease.leaseKey(), e); } catch (DynamoDbException | TimeoutException e) { throw convertAndRethrowExceptions("update", lease.leaseKey(), e); } 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 4523bada..832118dc 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 @@ -192,6 +192,16 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { return result; } + @Override + public Map getDynamoExistantExpectation() { + Map result = new HashMap<>(); + + ExpectedAttributeValue expectedAV = ExpectedAttributeValue.builder().exists(true).build(); + result.put(LEASE_KEY_KEY, expectedAV); + + return result; + } + @Override public Map getDynamoLeaseCounterUpdate(final Lease lease) { return getDynamoLeaseCounterUpdate(lease.leaseCounter()); From b7a5d7968b813523ad44ab16345d297b6f655862 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 19:31:52 -0400 Subject: [PATCH 156/159] Removing unnecessary code --- .../software/amazon/kinesis/coordinator/Scheduler.java | 7 +------ 1 file changed, 1 insertion(+), 6 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 db9cc145..ed2f889c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/coordinator/Scheduler.java @@ -348,12 +348,7 @@ public class Scheduler implements Runnable { log.info("Skipping shard sync per configuration setting (and lease table is not empty)"); } - if (!leaseCleanupManager.isRunning()) { - log.info("Starting LeaseCleanupManager."); - leaseCleanupManager.start(); - } else { - log.info("LeaseCleanupManager is already running. No need to start it"); - } + leaseCleanupManager.start(); // If we reach this point, then we either skipped the lease sync or did not have any exception // for any of the shard sync in the previous attempt. From e2a1d71053bb6223daea40d95662d954d8578334 Mon Sep 17 00:00:00 2001 From: Ashwin Giridharan Date: Mon, 27 Jul 2020 16:37:40 -0700 Subject: [PATCH 157/159] Update exception msg --- .../java/software/amazon/kinesis/leases/LeaseSerializer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java index 09d2280e..ff72891a 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java @@ -85,7 +85,7 @@ public interface LeaseSerializer { * @return the attribute value map asserting that a lease does exist. */ default Map getDynamoExistantExpectation() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("DynamoExistantExpectation is not implemented"); } /** From a48141682cca42a5026d34543e0a7a161dc7e6b6 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 27 Jul 2020 19:39:46 -0400 Subject: [PATCH 158/159] Moving debug to info logs to startup --- .../software/amazon/kinesis/leases/LeaseCleanupManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 0fb9ccd6..6e3104ae 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 @@ -82,14 +82,14 @@ public class LeaseCleanupManager { */ public void start() { if (!isRunning) { - log.debug("Starting lease cleanup thread."); + log.info("Starting lease cleanup thread."); completedLeaseStopwatch.reset().start(); garbageLeaseStopwatch.reset().start(); deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, TimeUnit.MILLISECONDS); isRunning = true; } else { - log.debug("Lease cleanup thread already running, no need to start."); + log.info("Lease cleanup thread already running, no need to start."); } } From 27494ea0438b9f8efb59f3ba98a5c35ae14b7dbb Mon Sep 17 00:00:00 2001 From: Renju Radhakrishnan Date: Tue, 28 Jul 2020 15:18:40 -0700 Subject: [PATCH 159/159] Set expectation criteria to use leaseKey (#88) * Set expectation criteria to use leaseKey * Set exists to true * Add integration tests for leaseMetaInfo update --- .../kinesis/leases/LeaseSerializer.java | 5 +- .../dynamodb/DynamoDBLeaseRefresher.java | 29 +++++++-- .../dynamodb/DynamoDBLeaseSerializer.java | 7 ++- ...DynamoDBLeaseRefresherIntegrationTest.java | 61 ++++++++++++++----- 4 files changed, 78 insertions(+), 24 deletions(-) diff --git a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java index ff72891a..f36f5a66 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/LeaseSerializer.java @@ -16,8 +16,6 @@ package software.amazon.kinesis.leases; import java.util.Collection; import java.util.Map; - - import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; @@ -82,9 +80,10 @@ public interface LeaseSerializer { Map getDynamoNonexistantExpectation(); /** + * @param leaseKey * @return the attribute value map asserting that a lease does exist. */ - default Map getDynamoExistantExpectation() { + default Map getDynamoExistentExpectation(String leaseKey) { throw new UnsupportedOperationException("DynamoExistantExpectation is not implemented"); } 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 98f75144..fb39e80f 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresher.java @@ -14,6 +14,8 @@ */ package software.amazon.kinesis.leases.dynamodb; +import com.google.common.collect.ImmutableMap; + import java.time.Duration; import java.util.ArrayList; import java.util.List; @@ -21,12 +23,30 @@ import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; - -import com.google.common.collect.ImmutableMap; import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; -import software.amazon.awssdk.services.dynamodb.model.*; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.AttributeValueUpdate; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException; +import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DeleteItemRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.model.DynamoDbException; +import software.amazon.awssdk.services.dynamodb.model.GetItemRequest; +import software.amazon.awssdk.services.dynamodb.model.GetItemResponse; +import software.amazon.awssdk.services.dynamodb.model.LimitExceededException; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughput; +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughputExceededException; +import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; +import software.amazon.awssdk.services.dynamodb.model.ResourceInUseException; +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; +import software.amazon.awssdk.services.dynamodb.model.ScanRequest; +import software.amazon.awssdk.services.dynamodb.model.ScanResponse; +import software.amazon.awssdk.services.dynamodb.model.TableStatus; +import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.kinesis.annotations.KinesisClientInternalApi; import software.amazon.kinesis.common.FutureUtils; @@ -41,7 +61,6 @@ import software.amazon.kinesis.leases.exceptions.InvalidStateException; import software.amazon.kinesis.leases.exceptions.ProvisionedThroughputException; import software.amazon.kinesis.retrieval.AWSExceptionManager; import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import software.amazon.awssdk.services.dynamodb.model.BillingMode; /** * An implementation of {@link LeaseRefresher} that uses DynamoDB. @@ -684,7 +703,7 @@ public class DynamoDBLeaseRefresher implements LeaseRefresher { exceptionManager.add(ConditionalCheckFailedException.class, t -> t); Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); UpdateItemRequest request = UpdateItemRequest.builder().tableName(table).key(serializer.getDynamoHashKey(lease)) - .expected(serializer.getDynamoExistantExpectation()) + .expected(serializer.getDynamoExistentExpectation(lease.leaseKey())) .attributeUpdates(updates).build(); try { try { 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 832118dc..64a7840c 100644 --- a/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java +++ b/amazon-kinesis-client/src/main/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseSerializer.java @@ -193,10 +193,13 @@ public class DynamoDBLeaseSerializer implements LeaseSerializer { } @Override - public Map getDynamoExistantExpectation() { + public Map getDynamoExistentExpectation(String leaseKey) { Map result = new HashMap<>(); - ExpectedAttributeValue expectedAV = ExpectedAttributeValue.builder().exists(true).build(); + ExpectedAttributeValue expectedAV = ExpectedAttributeValue.builder() + .exists(true) + .value(DynamoUtils.createAttributeValue(leaseKey)) + .build(); result.put(LEASE_KEY_KEY, expectedAV); return result; 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 75431866..1b2fa78a 100644 --- a/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java +++ b/amazon-kinesis-client/src/test/java/software/amazon/kinesis/leases/dynamodb/DynamoDBLeaseRefresherIntegrationTest.java @@ -14,6 +14,21 @@ */ package software.amazon.kinesis.leases.dynamodb; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; +import software.amazon.awssdk.services.kinesis.model.HashKeyRange; +import software.amazon.kinesis.common.HashKeyRangeForLease; +import software.amazon.kinesis.leases.Lease; +import software.amazon.kinesis.leases.LeaseIntegrationTest; +import software.amazon.kinesis.leases.UpdateField; +import software.amazon.kinesis.leases.exceptions.LeasingException; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -24,20 +39,6 @@ import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.verify; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.runners.MockitoJUnitRunner; - -import software.amazon.kinesis.leases.Lease; -import software.amazon.kinesis.leases.LeaseIntegrationTest; -import software.amazon.kinesis.leases.exceptions.LeasingException; - @RunWith(MockitoJUnitRunner.class) public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest { @@ -102,6 +103,38 @@ public class DynamoDBLeaseRefresherIntegrationTest extends LeaseIntegrationTest assertNull(actual); } + /** + * Tests leaseRefresher.updateLeaseWithMetaInfo() when the lease is deleted before updating it with meta info + */ + @Test + public void testDeleteLeaseThenUpdateLeaseWithMetaInfo() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + Lease lease = builder.withLease("1").build().get("1"); + final String leaseKey = lease.leaseKey(); + leaseRefresher.deleteLease(lease); + leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + final Lease deletedLease = leaseRefresher.getLease(leaseKey); + Assert.assertNull(deletedLease); + } + + /** + * Tests leaseRefresher.updateLeaseWithMetaInfo() on hashKeyRange update + */ + @Test + public void testUpdateLeaseWithMetaInfo() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher); + Lease lease = builder.withLease("1").build().get("1"); + final String leaseKey = lease.leaseKey(); + final HashKeyRangeForLease hashKeyRangeForLease = HashKeyRangeForLease.fromHashKeyRange(HashKeyRange.builder() + .startingHashKey("1") + .endingHashKey("2") + .build()); + lease.hashKeyRange(hashKeyRangeForLease); + leaseRefresher.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + final Lease updatedLease = leaseRefresher.getLease(leaseKey); + Assert.assertEquals(lease, updatedLease); + } + /** * Tests leaseRefresher.holdLease's success scenario. */