From c7cd2f1e75ae329131fd9c3279bebcaff74d7b13 Mon Sep 17 00:00:00 2001 From: Micah Jaffe Date: Thu, 30 Apr 2020 11:07:29 -0700 Subject: [PATCH 01/43] Migrate KinesisProxy to ListShardsWithFilter for empty lease table case --- .../lib/worker/KinesisShardSyncer.java | 55 +++++++++++++++++-- .../clientlibrary/proxies/IKinesisProxy.java | 12 ++++ .../clientlibrary/proxies/KinesisProxy.java | 40 +++++++++----- ...etricsCollectingKinesisProxyDecorator.java | 17 ++++++ .../proxies/KinesisLocalFileProxy.java | 11 ++++ 5 files changed, 118 insertions(+), 17 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index c23fd678..ac57f9cb 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -26,6 +26,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import com.amazonaws.services.kinesis.model.ShardFilter; +import com.amazonaws.services.kinesis.model.ShardFilterType; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -130,7 +132,14 @@ class KinesisShardSyncer implements ShardSyncer { boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - List latestShards = getShardList(kinesisProxy); + + // In the case where the lease table is empty, we want to synchronize the minimal amount of shards possible + // based on the given initial position. + // TODO: Implement shard list filtering on non-empty lease table case + final List latestShards = leaseManager.isLeaseTableEmpty() + ? getShardListAtInitialPosition(kinesisProxy, initialPosition) + : getCompleteShardList(kinesisProxy); + syncShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, latestShards); } @@ -156,7 +165,7 @@ class KinesisShardSyncer implements ShardSyncer { KinesisClientLibIOException { List shards; if(CollectionUtils.isNullOrEmpty(latestShards)) { - shards = getShardList(kinesisProxy); + shards = getCompleteShardList(kinesisProxy); } else { shards = latestShards; } @@ -345,7 +354,7 @@ class KinesisShardSyncer implements ShardSyncer { return shardIdToChildShardIdsMap; } - private List getShardList(IKinesisProxy kinesisProxy) throws KinesisClientLibIOException { + private List getCompleteShardList(IKinesisProxy kinesisProxy) throws KinesisClientLibIOException { List shards = kinesisProxy.getShardList(); if (shards == null) { throw new KinesisClientLibIOException( @@ -354,6 +363,44 @@ class KinesisShardSyncer implements ShardSyncer { return shards; } + private List getShardListAtInitialPosition(IKinesisProxy kinesisProxy, + InitialPositionInStreamExtended initialPosition) + throws KinesisClientLibIOException { + + final ShardFilter shardFilter = getShardFilterAtInitialPosition(initialPosition); + final List shards = kinesisProxy.getShardListWithFilter(shardFilter); + + if (shards == null) { + throw new KinesisClientLibIOException( + "Stream is not in ACTIVE OR UPDATING state - will retry getting the shard list."); + } + + return shards; + } + + private static ShardFilter getShardFilterAtInitialPosition(InitialPositionInStreamExtended initialPosition) { + ShardFilter shardFilter = new ShardFilter(); + + switch (initialPosition.getInitialPositionInStream()) { + case LATEST: + shardFilter = shardFilter.withType(ShardFilterType.AT_LATEST); + break; + case TRIM_HORIZON: + shardFilter = shardFilter.withType(ShardFilterType.AT_TRIM_HORIZON); + break; + case AT_TIMESTAMP: + shardFilter = shardFilter.withType(ShardFilterType.AT_TIMESTAMP) + .withTimestamp(initialPosition.getTimestamp()); + break; + default: + throw new IllegalArgumentException(initialPosition.getInitialPositionInStream() + + " is not a supported initial position in a Kinesis stream. Supported initial positions are" + + " AT_LATEST, AT_TRIM_HORIZON, and AT_TIMESTAMP."); + } + + return shardFilter; + } + /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. @@ -630,7 +677,7 @@ class KinesisShardSyncer implements ShardSyncer { if (!garbageLeases.isEmpty()) { LOG.info("Found " + garbageLeases.size() + " candidate leases for cleanup. Refreshing list of" + " Kinesis shards to pick up recent/latest shards"); - List currentShardList = getShardList(kinesisProxy); + List currentShardList = getCompleteShardList(kinesisProxy); Set currentKinesisShardIds = new HashSet<>(); for (Shard shard : currentShardList) { currentKinesisShardIds.add(shard.getShardId()); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java index 6e148969..7921a321 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/IKinesisProxy.java @@ -26,6 +26,7 @@ import com.amazonaws.services.kinesis.model.InvalidArgumentException; import com.amazonaws.services.kinesis.model.PutRecordResult; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardFilter; /** * Kinesis proxy interface. Operates on a single stream (set up at initialization). @@ -78,6 +79,17 @@ public interface IKinesisProxy { */ List getShardList() throws ResourceNotFoundException; + /** + * Fetch a subset shards defined for the stream using a filter on the ListShards API. This can be used to + * discover new shards and consume data from them, while limiting the total number of shards returned for + * performance or efficiency reasons. + * + * @param shardFilter currently supported filter types are AT_LATEST, AT_TRIM_HORIZON, AT_TIMESTAMP. + * @return List of all shards in the Kinesis stream. + * @throws ResourceNotFoundException The Kinesis stream was not found. + */ + List getShardListWithFilter(ShardFilter shardFilter) throws ResourceNotFoundException; + /** * Used to verify during ShardConsumer shutdown if the provided shardId is for a shard that has been closed. * @param shardId Id of the shard that needs to be verified. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java index 6717208b..0936bddb 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; +import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.util.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; @@ -309,7 +310,7 @@ public class KinesisProxy implements IKinesisProxyExtended { } } - private ListShardsResult listShards(final String nextToken) { + private ListShardsResult listShards(final ShardFilter shardFilter, final String nextToken) { final ListShardsRequest request = new ListShardsRequest(); request.setRequestCredentials(credentialsProvider.getCredentials()); if (StringUtils.isEmpty(nextToken)) { @@ -317,6 +318,11 @@ public class KinesisProxy implements IKinesisProxyExtended { } else { request.setNextToken(nextToken); } + + if (shardFilter != null) { + request.setShardFilter(shardFilter); + } + ListShardsResult result = null; LimitExceededException lastException = null; int remainingRetries = this.maxListShardsRetryAttempts; @@ -429,29 +435,37 @@ public class KinesisProxy implements IKinesisProxyExtended { */ @Override public synchronized List getShardList() { + return getShardListWithFilter(null); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized List getShardListWithFilter(ShardFilter shardFilter) { if (shardIterationState == null) { shardIterationState = new ShardIterationState(); } - + if (isKinesisClient) { ListShardsResult result; String nextToken = null; - + do { - result = listShards(nextToken); - + result = listShards(shardFilter, nextToken); + if (result == null) { /* - * If listShards ever returns null, we should bail and return null. This indicates the stream is not - * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream. - */ + * If listShards ever returns null, we should bail and return null. This indicates the stream is not + * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream. + */ return null; } else { shardIterationState.update(result.getShards()); nextToken = result.getNextToken(); } } while (StringUtils.isNotEmpty(result.getNextToken())); - + } else { DescribeStreamResult response; @@ -459,10 +473,10 @@ public class KinesisProxy implements IKinesisProxyExtended { response = getStreamInfo(shardIterationState.getLastShardId()); if (response == null) { - /* - * If getStreamInfo ever returns null, we should bail and return null. This indicates the stream is not - * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream. - */ + /* + * If getStreamInfo ever returns null, we should bail and return null. This indicates the stream is not + * in ACTIVE or UPDATING state and we may not have accurate/consistent information about the stream. + */ return null; } else { shardIterationState.update(response.getStreamDescription().getShards()); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java index 230ee710..fe6eb51e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/MetricsCollectingKinesisProxyDecorator.java @@ -28,6 +28,7 @@ import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; +import com.amazonaws.services.kinesis.model.ShardFilter; /** * IKinesisProxy implementation that wraps another implementation and collects metrics. @@ -179,6 +180,22 @@ public class MetricsCollectingKinesisProxyDecorator implements IKinesisProxy { } } + /** + * {@inheritDoc} + */ + @Override + public List getShardListWithFilter(ShardFilter shardFilter) throws ResourceNotFoundException { + long startTime = System.currentTimeMillis(); + boolean success = false; + try { + List response = other.getShardListWithFilter(shardFilter); + success = true; + return response; + } finally { + MetricsHelper.addSuccessAndLatency(getShardListMetric, startTime, success, MetricsLevel.DETAILED); + } + } + /** * {@inheritDoc} */ diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java index d78f5ca0..ed57eedf 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java @@ -33,6 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.util.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; @@ -425,6 +426,16 @@ public class KinesisLocalFileProxy implements IKinesisProxy { return shards; } + /** + * {@inheritDoc} + */ + @Override + public List getShardListWithFilter(ShardFilter shardFilter) throws ResourceNotFoundException { + List shards = new LinkedList(); + shards.addAll(shardList); + return shards; + } + /** * {@inheritDoc} */ From 550d7af5b193d475d35b4ba81390f635fb2d160c Mon Sep 17 00:00:00 2001 From: Micah Jaffe <31011877+micah-jaffe@users.noreply.github.com> Date: Mon, 4 May 2020 17:06:57 -0700 Subject: [PATCH 02/43] Add lease sync strategy for empty lease table (#24) * Add lease sync strategy for empty lease table * Fix ShardSyncer unit tests to reflect new empty lease table shard sync logic --- .../worker/EmptyLeaseTableSynchronizer.java | 98 +++++++ .../lib/worker/KinesisShardSyncer.java | 171 ++++-------- .../lib/worker/LeaseSynchronizer.java | 41 +++ .../NonEmptyLeaseTableSynchronizer.java | 165 ++++++++++++ .../lib/worker/ShardSyncerTest.java | 248 +++++++++++++++--- 5 files changed, 566 insertions(+), 157 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java new file mode 100644 index 00000000..3679f4dc --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/EmptyLeaseTableSynchronizer.java @@ -0,0 +1,98 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; +import com.amazonaws.services.kinesis.leases.impl.Lease; +import com.amazonaws.services.kinesis.model.Shard; +import lombok.AllArgsConstructor; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisShardSyncer.StartingSequenceNumberAndShardIdBasedComparator; + +/** + * Class to help create leases when the table is initially empty. + */ +@AllArgsConstructor +class EmptyLeaseTableSynchronizer implements LeaseSynchronizer { + + private static final Log LOG = LogFactory.getLog(EmptyLeaseTableSynchronizer.class); + + /** + * Determines how to create leases when the lease table is initially empty. For this, we read all shards where + * the KCL is reading from. For any shards which are closed, we will discover their child shards through GetRecords + * child shard information. + * + * @param shards + * @param currentLeases + * @param initialPosition + * @param inconsistentShardIds + * @return + */ + @Override + public List determineNewLeasesToCreate(List shards, + List currentLeases, + InitialPositionInStreamExtended initialPosition, + Set inconsistentShardIds) { + + final Map shardIdToShardMapOfAllKinesisShards = + KinesisShardSyncer.constructShardIdToShardMap(shards); + + currentLeases.forEach(lease -> LOG.debug("Existing lease: " + lease.getLeaseKey())); + + final List newLeasesToCreate = + getLeasesToCreateForOpenAndClosedShards(initialPosition, shards); + + final Comparator startingSequenceNumberComparator = + new StartingSequenceNumberAndShardIdBasedComparator(shardIdToShardMapOfAllKinesisShards); + + newLeasesToCreate.sort(startingSequenceNumberComparator); + return newLeasesToCreate; + } + + /** + * Helper method to create leases. For an empty lease table, we will be creating leases for all shards + * regardless of if they are open or closed. Closed shards will be unblocked via child shard information upon + * reaching SHARD_END. + */ + private List getLeasesToCreateForOpenAndClosedShards( + InitialPositionInStreamExtended initialPosition, + List shards) { + + final Map shardIdToNewLeaseMap = new HashMap<>(); + + for (Shard shard : shards) { + final String shardId = shard.getShardId(); + final KinesisClientLease lease = KinesisShardSyncer.newKCLLease(shard); + + final ExtendedSequenceNumber checkpoint = KinesisShardSyncer.convertToCheckpoint(initialPosition); + lease.setCheckpoint(checkpoint); + + LOG.debug("Need to create a lease for shard with shardId " + shardId); + shardIdToNewLeaseMap.put(shardId, lease); + } + + return new ArrayList(shardIdToNewLeaseMap.values()); + } +} \ No newline at end of file diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index ac57f9cb..13a52355 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -26,6 +26,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import com.amazonaws.services.kinesis.leases.impl.Lease; +import com.amazonaws.services.kinesis.leases.impl.LeaseManager; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; import com.amazonaws.util.CollectionUtils; @@ -45,6 +47,8 @@ import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.amazonaws.services.kinesis.model.Shard; +import javax.annotation.Nullable; + /** * 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). @@ -111,7 +115,8 @@ class KinesisShardSyncer implements ShardSyncer { boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards, List latestShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, latestShards); + syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, latestShards, leaseManager.isLeaseTableEmpty()); } /** @@ -136,11 +141,13 @@ class KinesisShardSyncer implements ShardSyncer { // In the case where the lease table is empty, we want to synchronize the minimal amount of shards possible // based on the given initial position. // TODO: Implement shard list filtering on non-empty lease table case - final List latestShards = leaseManager.isLeaseTableEmpty() + final boolean isLeaseTableEmpty = leaseManager.isLeaseTableEmpty(); + final List latestShards = isLeaseTableEmpty ? getShardListAtInitialPosition(kinesisProxy, initialPosition) : getCompleteShardList(kinesisProxy); - syncShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, latestShards); + syncShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, latestShards, isLeaseTableEmpty); } /** @@ -159,10 +166,15 @@ class KinesisShardSyncer implements ShardSyncer { */ // CHECKSTYLE:OFF CyclomaticComplexity private synchronized void syncShardLeases(IKinesisProxy kinesisProxy, - ILeaseManager leaseManager, InitialPositionInStreamExtended initialPosition, - boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards, List latestShards) + ILeaseManager leaseManager, + InitialPositionInStreamExtended initialPosition, + boolean cleanupLeasesOfCompletedShards, + boolean ignoreUnexpectedChildShards, + List latestShards, + boolean isLeaseTableEmpty) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + List shards; if(CollectionUtils.isNullOrEmpty(latestShards)) { shards = getCompleteShardList(kinesisProxy); @@ -178,11 +190,16 @@ class KinesisShardSyncer implements ShardSyncer { assertAllParentShardsAreClosed(inconsistentShardIds); } - List currentLeases = leaseManager.listLeases(); + // Determine which lease sync strategy to use based on the state of the lease table + final LeaseSynchronizer leaseSynchronizer = isLeaseTableEmpty + ? new EmptyLeaseTableSynchronizer() + : new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap); - List newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition, - inconsistentShardIds); + final List currentLeases = leaseManager.listLeases(); + final List newLeasesToCreate = determineNewLeasesToCreate(leaseSynchronizer, shards, + currentLeases, initialPosition, inconsistentShardIds); LOG.debug("Num new leases to create: " + newLeasesToCreate.size()); + for (KinesisClientLease lease : newLeasesToCreate) { long startTimeMillis = System.currentTimeMillis(); boolean success = false; @@ -326,7 +343,7 @@ class KinesisShardSyncer implements ShardSyncer { * @param shardIdToShardMap * @return */ - Map> constructShardIdToChildShardIdsMap(Map shardIdToShardMap) { + static Map> constructShardIdToChildShardIdsMap(Map shardIdToShardMap) { Map> shardIdToChildShardIdsMap = new HashMap<>(); for (Map.Entry entry : shardIdToShardMap.entrySet()) { String shardId = entry.getKey(); @@ -405,42 +422,8 @@ class KinesisShardSyncer implements ShardSyncer { * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. * - * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, - * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): - * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. - * If not, set checkpoint of the shard to the initial position specified by the client. - * To check if we need to create leases for ancestors, we use the following rules: - * * If we began (or will begin) processing data for a shard, then we must reach end of that shard before - * we begin processing data from any of its descendants. - * * A shard does not start processing data until data from all its parents has been processed. - * Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create - * leases corresponding to both the parents - the parent shard which is not a descendant will have - * its checkpoint set to Latest. - * - * We assume that if there is an existing lease for a shard, then either: - * * we have previously created a lease for its parent (if it was needed), or - * * the parent shard has expired. - * - * For example: - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | / \ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) - * New leases to create: (2, 6, 7, 8, 9, 10) - * - * The leases returned are sorted by the starting sequence number - following the same order - * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail - * before creating all the leases. - * - * If a shard has no existing lease, is open, and is a descendant of a parent which is still open, we ignore it - * here; this happens when the list of shards is inconsistent, which could be due to pagination delay for very - * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only - * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. - * - * + * @param leaseSynchronizer determines the strategy to use when updating leases based on the current state of + * the lease table (empty vs. non-empty) * @param shards List of all shards in Kinesis (we'll create new leases based on this set) * @param currentLeases List of current leases * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that @@ -448,85 +431,27 @@ class KinesisShardSyncer implements ShardSyncer { * @param inconsistentShardIds Set of child shard ids having open parents. * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ - List determineNewLeasesToCreate(List shards, List currentLeases, - InitialPositionInStreamExtended initialPosition, Set inconsistentShardIds) { - Map shardIdToNewLeaseMap = new HashMap(); - Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); + List determineNewLeasesToCreate(LeaseSynchronizer leaseSynchronizer, + List shards, + List currentLeases, + InitialPositionInStreamExtended initialPosition, + Set inconsistentShardIds) { - Set shardIdsOfCurrentLeases = new HashSet(); - for (KinesisClientLease lease : currentLeases) { - shardIdsOfCurrentLeases.add(lease.getLeaseKey()); - LOG.debug("Existing lease: " + lease); - } - - List openShards = getOpenShards(shards); - Map memoizationContext = new HashMap<>(); - - // Iterate over the open shards and find those that don't have any lease entries. - for (Shard shard : openShards) { - String shardId = shard.getShardId(); - LOG.debug("Evaluating leases for open shard " + shardId + " and its ancestors."); - if (shardIdsOfCurrentLeases.contains(shardId)) { - LOG.debug("Lease for shardId " + shardId + " already exists. Not creating a lease"); - } else if (inconsistentShardIds.contains(shardId)) { - LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease"); - } else { - LOG.debug("Need to create a lease for shardId " + shardId); - KinesisClientLease newLease = newKCLLease(shard); - boolean isDescendant = checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, - shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, - memoizationContext); - - /** - * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the - * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a - * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side - * timestamp at or after the specified initial position timestamp. - * - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * - * Current leases: empty set - * - * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with - * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to - * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin - * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases - * would then be deleted since they won't have records with server-side timestamp at/after 206. And - * after that we will begin processing the descendant shards with epoch at/after 206 and we will - * return the records that meet the timestamp requirement for these shards. - */ - if (isDescendant && !initialPosition.getInitialPositionInStream() - .equals(InitialPositionInStream.AT_TIMESTAMP)) { - newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - newLease.setCheckpoint(convertToCheckpoint(initialPosition)); - } - LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint()); - shardIdToNewLeaseMap.put(shardId, newLease); - } - } - - List newLeasesToCreate = new ArrayList(); - newLeasesToCreate.addAll(shardIdToNewLeaseMap.values()); - Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMapOfAllKinesisShards); - Collections.sort(newLeasesToCreate, startingSequenceNumberComparator); - return newLeasesToCreate; + return leaseSynchronizer.determineNewLeasesToCreate(shards, currentLeases, initialPosition, + inconsistentShardIds); } /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. */ - List determineNewLeasesToCreate(List shards, List currentLeases, - InitialPositionInStreamExtended initialPosition) { + List determineNewLeasesToCreate(LeaseSynchronizer leaseSynchronizer, + List shards, + List currentLeases, + InitialPositionInStreamExtended initialPosition) { + Set inconsistentShardIds = new HashSet(); - return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds); + return determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition, inconsistentShardIds); } /** @@ -545,7 +470,7 @@ class KinesisShardSyncer implements ShardSyncer { * @return true if the shard is a descendant of any current shard (lease already exists) */ // CHECKSTYLE:OFF CyclomaticComplexity - boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId, + static boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId, InitialPositionInStreamExtended initialPosition, Set shardIdsOfCurrentLeases, Map shardIdToShardMapOfAllKinesisShards, Map shardIdToLeaseMapOfNewShards, Map memoizationContext) { @@ -630,7 +555,7 @@ class KinesisShardSyncer implements ShardSyncer { * @param shardIdToShardMapOfAllKinesisShards ShardId->Shard map containing all shards obtained via DescribeStream. * @return Set of parentShardIds */ - Set getParentShardIds(Shard shard, Map shardIdToShardMapOfAllKinesisShards) { + static Set getParentShardIds(Shard shard, Map shardIdToShardMapOfAllKinesisShards) { Set parentShardIds = new HashSet(2); String parentShardId = shard.getParentShardId(); if ((parentShardId != null) && shardIdToShardMapOfAllKinesisShards.containsKey(parentShardId)) { @@ -794,7 +719,7 @@ class KinesisShardSyncer implements ShardSyncer { * @param shard * @return */ - KinesisClientLease newKCLLease(Shard shard) { + static KinesisClientLease newKCLLease(Shard shard) { KinesisClientLease newLease = new KinesisClientLease(); newLease.setLeaseKey(shard.getShardId()); List parentShardIds = new ArrayList(2); @@ -816,7 +741,7 @@ class KinesisShardSyncer implements ShardSyncer { * @param shards List of shards * @return ShardId->Shard map */ - Map constructShardIdToShardMap(List shards) { + static Map constructShardIdToShardMap(List shards) { Map shardIdToShardMap = new HashMap(); for (Shard shard : shards) { shardIdToShardMap.put(shard.getShardId(), shard); @@ -831,7 +756,7 @@ class KinesisShardSyncer implements ShardSyncer { * @param allShards All shards returved via DescribeStream. We assume this to represent a consistent shard list. * @return List of open shards (shards at the tip of the stream) - may include shards that are not yet active. */ - List getOpenShards(List allShards) { + static List getOpenShards(List allShards) { List openShards = new ArrayList(); for (Shard shard : allShards) { String endingSequenceNumber = shard.getSequenceNumberRange().getEndingSequenceNumber(); @@ -843,7 +768,7 @@ class KinesisShardSyncer implements ShardSyncer { return openShards; } - private ExtendedSequenceNumber convertToCheckpoint(InitialPositionInStreamExtended position) { + static ExtendedSequenceNumber convertToCheckpoint(InitialPositionInStreamExtended position) { ExtendedSequenceNumber checkpoint = null; if (position.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)) { @@ -860,7 +785,7 @@ class KinesisShardSyncer implements ShardSyncer { /** Helper class to compare leases based on starting sequence number of the corresponding shards. * */ - private static class StartingSequenceNumberAndShardIdBasedComparator implements Comparator, + static class StartingSequenceNumberAndShardIdBasedComparator implements Comparator, Serializable { private static final long serialVersionUID = 1L; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java new file mode 100644 index 00000000..441249d1 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseSynchronizer.java @@ -0,0 +1,41 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; +import com.amazonaws.services.kinesis.model.Shard; + +import java.util.List; +import java.util.Set; + +/** + * Interface used by {@link KinesisShardSyncer} to determine how to create new leases based on the current state + * of the lease table (i.e. whether the lease table is empty or non-empty). + */ +interface LeaseSynchronizer { + + /** + * Determines how to create leases. + * @param shards + * @param currentLeases + * @param initialPosition + * @param inconsistentShardIds + * @return + */ + List determineNewLeasesToCreate(List shards, + List currentLeases, + InitialPositionInStreamExtended initialPosition, + Set inconsistentShardIds); +} \ No newline at end of file diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java new file mode 100644 index 00000000..53c42980 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java @@ -0,0 +1,165 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; +import com.amazonaws.services.kinesis.leases.impl.Lease; +import com.amazonaws.services.kinesis.model.Shard; +import lombok.AllArgsConstructor; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * TODO - non-empty lease table sync story + */ +@AllArgsConstructor +class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { + + private static final Log LOG = LogFactory.getLog(NonEmptyLeaseTableSynchronizer.class); + + private final Map shardIdToShardMap; + private final Map> shardIdToChildShardIdsMap; + + /** + * Determine new leases to create and their initial checkpoint. + * Note: Package level access only for testing purposes. + * + * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, + * determine if it is a 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 shards List of all shards in Kinesis (we'll create new leases based on this set) + * @param currentLeases List of current leases + * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that + * location in the shard (when an application starts up for the first time - and there are no checkpoints). + * @param inconsistentShardIds Set of child shard ids having open parents. + * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard + */ + @Override + public List determineNewLeasesToCreate(List shards, + List currentLeases, + InitialPositionInStreamExtended initialPosition, + Set inconsistentShardIds) { + + Map shardIdToNewLeaseMap = new HashMap<>(); + Map shardIdToShardMapOfAllKinesisShards = KinesisShardSyncer.constructShardIdToShardMap(shards); + + Set shardIdsOfCurrentLeases = new HashSet(); + for (Lease lease : currentLeases) { + shardIdsOfCurrentLeases.add(lease.getLeaseKey()); + LOG.debug("Existing lease: " + lease); + } + + List openShards = KinesisShardSyncer.getOpenShards(shards); + Map memoizationContext = new HashMap<>(); + + // Iterate over the open shards and find those that don't have any lease entries. + for (Shard shard : openShards) { + String shardId = shard.getShardId(); + LOG.debug("Evaluating leases for open shard " + shardId + " and its ancestors."); + if (shardIdsOfCurrentLeases.contains(shardId)) { + LOG.debug("Lease for shardId " + shardId + " already exists. Not creating a lease"); + } else if (inconsistentShardIds.contains(shardId)) { + LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease"); + } else { + LOG.debug("Need to create a lease for shardId " + shardId); + KinesisClientLease newLease = KinesisShardSyncer.newKCLLease(shard); + boolean isDescendant = KinesisShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, + initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, + shardIdToNewLeaseMap, memoizationContext); + + /** + * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the + * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a + * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side + * timestamp at or after the specified initial position timestamp. + * + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: empty set + * + * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with + * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to + * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin + * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases + * would then be deleted since they won't have records with server-side timestamp at/after 206. And + * after that we will begin processing the descendant shards with epoch at/after 206 and we will + * return the records that meet the timestamp requirement for these shards. + */ + if (isDescendant && !initialPosition.getInitialPositionInStream() + .equals(InitialPositionInStream.AT_TIMESTAMP)) { + newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } else { + newLease.setCheckpoint(KinesisShardSyncer.convertToCheckpoint(initialPosition)); + } + LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint()); + shardIdToNewLeaseMap.put(shardId, newLease); + } + } + + List newLeasesToCreate = new ArrayList<>(); + newLeasesToCreate.addAll(shardIdToNewLeaseMap.values()); + Comparator startingSequenceNumberComparator = new KinesisShardSyncer.StartingSequenceNumberAndShardIdBasedComparator( + shardIdToShardMapOfAllKinesisShards); + Collections.sort(newLeasesToCreate, startingSequenceNumberComparator); + return newLeasesToCreate; + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index 2fce50a7..83dcd4af 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -18,17 +18,21 @@ import java.io.File; import java.io.IOException; import java.math.BigInteger; import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; -import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; -import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient; import com.amazonaws.services.dynamodbv2.local.embedded.DynamoDBEmbedded; +import com.amazonaws.services.kinesis.leases.impl.Lease; +import com.amazonaws.services.kinesis.model.ShardFilter; +import com.amazonaws.services.kinesis.model.ShardFilterType; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.After; @@ -56,6 +60,11 @@ import com.amazonaws.services.kinesis.model.Shard; import junit.framework.Assert; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; + /** * */ @@ -69,9 +78,9 @@ public class ShardSyncerTest { private static final InitialPositionInStreamExtended INITIAL_POSITION_AT_TIMESTAMP = InitialPositionInStreamExtended.newInitialPositionAtTimestamp(new Date(1000L)); private final boolean cleanupLeasesOfCompletedShards = true; - AmazonDynamoDB ddbClient = DynamoDBEmbedded.create().amazonDynamoDB(); - LeaseManager leaseManager = new KinesisClientLeaseManager("tempTestTable", ddbClient, KinesisClientLibConfiguration.DEFAULT_DDB_BILLING_MODE); private static final int EXPONENT = 128; + AmazonDynamoDB ddbClient = DynamoDBEmbedded.create().amazonDynamoDB(); + private LeaseManager leaseManager = new KinesisClientLeaseManager("tempTestTable", ddbClient, KinesisClientLibConfiguration.DEFAULT_DDB_BILLING_MODE); protected static final KinesisLeaseCleanupValidator leaseCleanupValidator = new KinesisLeaseCleanupValidator(); private static final KinesisShardSyncer shardSyncer = new KinesisShardSyncer(leaseCleanupValidator); /** @@ -120,8 +129,9 @@ public class ShardSyncerTest { public final void testDetermineNewLeasesToCreateNoShards() { List shards = new ArrayList(); List leases = new ArrayList(); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, leases); - Assert.assertTrue(shardSyncer.determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST).isEmpty()); + Assert.assertTrue(shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, leases, INITIAL_POSITION_LATEST).isEmpty()); } /** @@ -139,8 +149,10 @@ public class ShardSyncerTest { String shardId1 = "shardId-1"; shards.add(ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); Assert.assertEquals(2, newLeases.size()); Set expectedLeaseShardIds = new HashSet(); expectedLeaseShardIds.add(shardId0); @@ -151,7 +163,7 @@ public class ShardSyncerTest { } /** - * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but one of + * Test determineNewLeasesToCreate() where there is one lease and no resharding operations have been performed, but one of * the shards was marked as inconsistent. */ @Test @@ -169,11 +181,18 @@ public class ShardSyncerTest { String shardId2 = "shardId-2"; shards.add(ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); + String shardIdWithLease = "shardId-3"; + shards.add(ShardObjectHelper.newShard(shardIdWithLease, shardIdWithLease, null, sequenceRange)); + + currentLeases.add(newLease(shardIdWithLease)); + Set inconsistentShardIds = new HashSet(); inconsistentShardIds.add(shardId2); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); Assert.assertEquals(2, newLeases.size()); Set expectedLeaseShardIds = new HashSet(); expectedLeaseShardIds.add(shardId0); @@ -216,14 +235,10 @@ public class ShardSyncerTest { } /** - * @throws KinesisClientLibIOException - * @throws DependencyException - * @throws InvalidStateException - * @throws ProvisionedThroughputException - * @throws IOException + * All open and closed shards within stream's retention period should be sync'ed when lease table is empty. */ @Test - public final void testCheckAndCreateLeasesForNewShardsAtLatest() + public final void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { List shards = constructShardListForGraphA(); @@ -231,6 +246,38 @@ public class ShardSyncerTest { dataFile.deleteOnExit(); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST, + cleanupLeasesOfCompletedShards, false, shards); + List newLeases = leaseManager.listLeases(); + Set expectedLeaseShardIds = new HashSet(); + for (int i = 0; i < 11; i++) { + expectedLeaseShardIds.add("shardId-" + i); + } + Assert.assertEquals(expectedLeaseShardIds.size(), newLeases.size()); + for (KinesisClientLease lease1 : newLeases) { + Assert.assertTrue(expectedLeaseShardIds.contains(lease1.getLeaseKey())); + Assert.assertEquals(ExtendedSequenceNumber.LATEST, lease1.getCheckpoint()); + } + dataFile.delete(); + } + + /** + * We should only create leases for shards at LATEST when lease table is not empty. + */ + @Test + public final void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() + throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, + IOException { + List shards = constructShardListForGraphA(); + File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); + dataFile.deleteOnExit(); + IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); + + // shardId-10 exists at LATEST - create a lease for it + KinesisClientLease lease = newLease("shardId-10"); + lease.setCheckpoint(ExtendedSequenceNumber.LATEST); + leaseManager.createLeaseIfNotExists(lease); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST, cleanupLeasesOfCompletedShards, false, shards); List newLeases = leaseManager.listLeases(); @@ -353,6 +400,13 @@ public class ShardSyncerTest { File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); dataFile.deleteOnExit(); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); + + // Create a dummy lease in the lease table - otherwise leaseManager will create leases for all shards if + // lease table is empty. + KinesisClientLease lease = newLease("shardId-1000"); + lease.setCheckpoint(ExtendedSequenceNumber.LATEST); + leaseManager.createLeaseIfNotExists(lease); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST, cleanupLeasesOfCompletedShards, true, shards); List newLeases = leaseManager.listLeases(); @@ -695,8 +749,9 @@ public class ShardSyncerTest { initialPositions.add(INITIAL_POSITION_TRIM_HORIZON); for (InitialPositionInStreamExtended initialPosition : initialPositions) { + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, initialPosition); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, initialPosition); Assert.assertEquals(2, newLeases.size()); Set expectedLeaseShardIds = new HashSet(); expectedLeaseShardIds.add(shardId0); @@ -710,27 +765,73 @@ public class ShardSyncerTest { } /** - * Test determineNewLeasesToCreate() - 1 closed and 1 open shard (ignore closed shard) + * Test determineNewLeasesToCreate() - 1 closed and 1 open shard (ignore closed shard), 1 shard with a lease + * already in lease table. If lease table is non-empty, closed shards should be ignored. */ @Test - public final void testDetermineNewLeasesToCreateIgnoreClosedShard() { - List shards = new ArrayList(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateIgnoresClosedShardWithPartialLeaseTable() { + final List shardsWithoutLeases = new ArrayList(); + final List shardsWithLeases = new ArrayList(); + final List currentLeases = new ArrayList(); - shards.add(ShardObjectHelper.newShard("shardId-0", + shardsWithoutLeases.add(ShardObjectHelper.newShard("shardId-0", null, null, ShardObjectHelper.newSequenceNumberRange("303", "404"))); - String lastShardId = "shardId-1"; + final String lastShardId = "shardId-1"; + shardsWithoutLeases.add(ShardObjectHelper.newShard(lastShardId, + null, + null, + ShardObjectHelper.newSequenceNumberRange("405", null))); + + shardsWithLeases.add(ShardObjectHelper.newShard("shardId-2", + null, + null, + ShardObjectHelper.newSequenceNumberRange("202", "302"))); + currentLeases.add(newLease("shardId-2")); + + final List allShards = + Stream.concat(shardsWithLeases.stream(), shardsWithoutLeases.stream()).collect(Collectors.toList()); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(allShards, currentLeases); + + final List newLeases = + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shardsWithoutLeases, currentLeases, INITIAL_POSITION_LATEST); + Assert.assertEquals(1, newLeases.size()); + Assert.assertEquals(lastShardId, newLeases.get(0).getLeaseKey()); + } + + /** + * Test determineNewLeasesToCreate() - 1 closed and 1 open shard. Since lease table is empty, we should create + * leases for all shards, regardless if they are open or closed. + */ + @Test + public final void testDetermineNewLeasesToCreateDoesntIgnoreClosedShardWithEmptyLeaseTable() { + final List shards = new ArrayList<>(); + final List currentLeases = new ArrayList<>(); + + final String firstShardId = "shardId-0"; + shards.add(ShardObjectHelper.newShard(firstShardId, + null, + null, + ShardObjectHelper.newSequenceNumberRange("303", "404"))); + final String lastShardId = "shardId-1"; shards.add(ShardObjectHelper.newShard(lastShardId, null, null, ShardObjectHelper.newSequenceNumberRange("405", null))); - List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); - Assert.assertEquals(1, newLeases.size()); - Assert.assertEquals(lastShardId, newLeases.get(0).getLeaseKey()); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + + final List newLeases = + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); + Assert.assertEquals(2, newLeases.size()); + + final Set expectedLeaseShardIds = new HashSet<>(); + expectedLeaseShardIds.add(firstShardId); + expectedLeaseShardIds.add(lastShardId); + for (KinesisClientLease lease : newLeases) { + Assert.assertTrue(expectedLeaseShardIds.contains(lease.getLeaseKey())); + } } /** @@ -752,8 +853,10 @@ public class ShardSyncerTest { currentLeases.add(newLease("shardId-4")); currentLeases.add(newLease("shardId-5")); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); Map expectedShardIdCheckpointMap = new HashMap(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); @@ -790,8 +893,10 @@ public class ShardSyncerTest { currentLeases.add(newLease("shardId-5")); currentLeases.add(newLease("shardId-7")); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); Map expectedShardIdCheckpointMap = new HashMap(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); @@ -826,8 +931,10 @@ public class ShardSyncerTest { currentLeases.add(newLease("shardId-4")); currentLeases.add(newLease("shardId-5")); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); Map expectedShardIdCheckpointMap = new HashMap(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); @@ -866,8 +973,10 @@ public class ShardSyncerTest { currentLeases.add(newLease("shardId-5")); currentLeases.add(newLease("shardId-7")); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); Map expectedShardIdCheckpointMap = new HashMap(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); @@ -895,8 +1004,9 @@ public class ShardSyncerTest { public final void testDetermineNewLeasesToCreateGraphBNoInitialLeasesTrim() { List shards = constructShardListForGraphB(); List currentLeases = new ArrayList(); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); Map expectedShardIdCheckpointMap = new HashMap(); for (int i = 0; i < 11; i++) { @@ -927,13 +1037,14 @@ public class ShardSyncerTest { List shards = constructShardListForGraphA(); List currentLeases = new ArrayList(); - currentLeases.add(newLease("shardId-3")); currentLeases.add(newLease("shardId-4")); currentLeases.add(newLease("shardId-5")); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); Map expectedShardIdCheckpointMap = new HashMap(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); @@ -971,8 +1082,10 @@ public class ShardSyncerTest { currentLeases.add(newLease("shardId-5")); currentLeases.add(newLease("shardId-7")); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); + List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); Map expectedShardIdCheckpointMap = new HashMap(); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); @@ -998,8 +1111,9 @@ public class ShardSyncerTest { public final void testDetermineNewLeasesToCreateGraphBNoInitialLeasesAtTimestamp() { List shards = constructShardListForGraphB(); List currentLeases = new ArrayList(); + final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); List newLeases = - shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); Map expectedShardIdCheckpointMap = new HashMap(); for (int i = 0; i < shards.size(); i++) { @@ -1664,6 +1778,51 @@ public class ShardSyncerTest { testAssertShardCoveredOrAbsentTestIncompleteSplit(hashKeyRange, childHashKeyRange1, childHashKeyRange2); } + /** + * Tests that when reading from TIP, we use the AT_LATEST shard filter + * @throws Exception + */ + @Test + public final void testEmptyLeaseTableBootstrapUsesShardFilterWithAtLatest() throws Exception { + ShardFilter shardFilter = new ShardFilter().withType(ShardFilterType.AT_LATEST); + testEmptyLeaseTableUsesListShardsWithFilter(INITIAL_POSITION_LATEST, shardFilter); + } + + /** + * Tests that when reading from TRIM, we use the TRIM_HORIZON shard filter + * @throws Exception + */ + @Test + public final void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTrimHorizon() throws Exception { + ShardFilter shardFilter = new ShardFilter().withType(ShardFilterType.AT_TRIM_HORIZON); + testEmptyLeaseTableUsesListShardsWithFilter(INITIAL_POSITION_TRIM_HORIZON, shardFilter); + } + + /** + * Tests that when reading from AT_TIMESTAMP, we use the AT_TIMESTAMP shard filter + * @throws Exception + */ + @Test + public final void testEmptyLeaseTableBootstrapUsesShardFilterWithAtTimestamp() throws Exception { + ShardFilter shardFilter = new ShardFilter().withType(ShardFilterType.AT_TIMESTAMP).withTimestamp(new Date(1000L)); + testEmptyLeaseTableUsesListShardsWithFilter(INITIAL_POSITION_AT_TIMESTAMP, shardFilter); + } + + private void testEmptyLeaseTableUsesListShardsWithFilter(InitialPositionInStreamExtended initialPosition, + ShardFilter shardFilter) throws Exception { + + final List shards = constructShardListForGraphA(); + final File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); + dataFile.deleteOnExit(); + final IKinesisProxy kinesisProxy = spy(new KinesisLocalFileProxy(dataFile.getAbsolutePath())); + + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPosition, + cleanupLeasesOfCompletedShards, false); + + verify(kinesisProxy, atLeastOnce()).getShardListWithFilter(shardFilter); + verify(kinesisProxy, never()).getShardList(); + } + private void testAssertShardCoveredOrAbsentTestIncompleteSplit(HashKeyRange parentHashKeyRange, HashKeyRange child1HashKeyRange, HashKeyRange child2HashKeyRange) @@ -1710,4 +1869,25 @@ public class ShardSyncerTest { return lease; } + /** + * Helper method to get appropriate LeaseSynchronizer based on available shards and current leases. If there are + * no current leases (empty lease table case), return EmptyLeaseTableSynchronizer. Else, return + * NonEmptyLeaseTableSynchronizer with appropriate lease mappings. + * + * @param shards + * @param currentLeases + * @return + */ + private LeaseSynchronizer getLeaseSynchronizer(List shards, List currentLeases) { + if (currentLeases.isEmpty()) { + return new EmptyLeaseTableSynchronizer(); + } + + final Map shardIdToShardMap = KinesisShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = + KinesisShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + + return new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap); + } + } From 45e76b0fd5c5da169de7d35955fe1ed606f59aa6 Mon Sep 17 00:00:00 2001 From: Micah Jaffe <31011877+micah-jaffe@users.noreply.github.com> Date: Thu, 7 May 2020 17:08:53 -0700 Subject: [PATCH 03/43] Add leader-elected shard sync on application bootstrap (#28) * Add leader-elected shard sync on application bootstrap * Add additional testing and address PR feedback * Remove runShardSync leader election from boostrap case * Remove random UUID workerId and update unit tests --- .../lib/worker/KinesisShardSyncer.java | 2 +- .../lib/worker/PeriodicShardSyncManager.java | 34 ++-- .../lib/worker/ShardEndShardSyncStrategy.java | 2 + .../clientlibrary/lib/worker/Worker.java | 150 +++++++++++++----- .../lib/worker/ShardSyncerTest.java | 9 +- .../clientlibrary/lib/worker/WorkerTest.java | 101 ++++++++++-- 6 files changed, 234 insertions(+), 64 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index 13a52355..21890663 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -177,7 +177,7 @@ class KinesisShardSyncer implements ShardSyncer { List shards; if(CollectionUtils.isNullOrEmpty(latestShards)) { - shards = getCompleteShardList(kinesisProxy); + shards = getShardListAtInitialPosition(kinesisProxy, initialPosition); } else { shards = latestShards; } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java index d129944f..87237e0b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java @@ -58,14 +58,31 @@ class PeriodicShardSyncManager { public synchronized TaskResult start() { if (!isRunning) { + final Runnable periodicShardSyncer = () -> { + try { + runShardSync(); + } catch (Throwable t) { + LOG.error("Error running shard sync.", t); + } + }; + shardSyncThreadPool - .scheduleWithFixedDelay(this::runShardSync, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, + .scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, TimeUnit.MILLISECONDS); isRunning = true; } return new TaskResult(null); } + /** + * Runs ShardSync once, without scheduling further periodic ShardSyncs. + * @return TaskResult from shard sync + */ + public synchronized TaskResult syncShardsOnce() { + LOG.info("Syncing shards once from worker " + workerId); + return metricsEmittingShardSyncTask.call(); + } + public void stop() { if (isRunning) { LOG.info(String.format("Shutting down leader decider on worker %s", workerId)); @@ -77,15 +94,12 @@ class PeriodicShardSyncManager { } private void runShardSync() { - try { - if (leaderDecider.isLeader(workerId)) { - LOG.debug(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); - metricsEmittingShardSyncTask.call(); - } else { - LOG.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); - } - } catch (Throwable t) { - LOG.error("Error during runShardSync.", t); + if (leaderDecider.isLeader(workerId)) { + LOG.debug(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); + metricsEmittingShardSyncTask.call(); + } else { + LOG.debug(String.format("WorkerId %s is not a leader, not running the shard sync task", workerId)); } + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java index 8077efcc..18fc4a08 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java @@ -42,6 +42,7 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy { @Override public TaskResult onWorkerInitialization() { + // TODO: Start leaderElectedPeriodicShardSyncManager in background LOG.debug(String.format("onWorkerInitialization is NoOp for ShardSyncStrategyType %s", getStrategyType().toString())); return new TaskResult(null); } @@ -65,6 +66,7 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy { @Override public void onWorkerShutDown() { + // TODO: Shut down leaderElectedPeriodicShardSyncManager LOG.debug(String.format("Stop is NoOp for ShardSyncStrategyType %s", getStrategyType().toString())); } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 9eabcffe..5f1b4fac 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -29,11 +29,15 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Consumer; +import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector; import com.amazonaws.services.kinesis.leases.impl.LeaseCoordinator; import com.amazonaws.services.kinesis.leases.impl.LeaseRenewer; @@ -88,9 +92,14 @@ public class Worker implements Runnable { private static final Log LOG = LogFactory.getLog(Worker.class); + // Default configs for periodic shard sync private static final int SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC = 0; private static final int MAX_INITIALIZATION_ATTEMPTS = 20; private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1; //Default for KCL. + static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; + static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; + static final long MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 30 * 1000L; + private static final WorkerStateChangeListener DEFAULT_WORKER_STATE_CHANGE_LISTENER = new NoOpWorkerStateChangeListener(); private static final LeaseCleanupValidator DEFAULT_LEASE_CLEANUP_VALIDATOR = new KinesisLeaseCleanupValidator(); private static final LeaseSelector DEFAULT_LEASE_SELECTOR = new GenericLeaseSelector(); @@ -147,6 +156,7 @@ public class Worker implements Runnable { // Periodic Shard Sync related fields private LeaderDecider leaderDecider; private ShardSyncStrategy shardSyncStrategy; + private PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager; /** * Constructor. @@ -406,7 +416,7 @@ public class Worker implements Runnable { config.getShardPrioritizationStrategy(), config.getRetryGetRecordsInSeconds(), config.getMaxGetRecordsThreadPool(), - DEFAULT_WORKER_STATE_CHANGE_LISTENER, DEFAULT_LEASE_CLEANUP_VALIDATOR, null /* leaderDecider */); + DEFAULT_WORKER_STATE_CHANGE_LISTENER, DEFAULT_LEASE_CLEANUP_VALIDATOR, null, null); // If a region name was explicitly specified, use it as the region for Amazon Kinesis and Amazon DynamoDB. if (config.getRegionName() != null) { @@ -467,7 +477,7 @@ public class Worker implements Runnable { shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization, Optional.empty(), Optional.empty(), DEFAULT_WORKER_STATE_CHANGE_LISTENER, - DEFAULT_LEASE_CLEANUP_VALIDATOR, null); + DEFAULT_LEASE_CLEANUP_VALIDATOR, null, null); } /** @@ -507,6 +517,10 @@ public class Worker implements Runnable { * Max number of threads in the getRecords thread pool. * @param leaseCleanupValidator * leaseCleanupValidator instance used to validate leases + * @param leaderDecider + * leaderDecider instance used elect shard sync leaders + * @param periodicShardSyncManager + * manages periodic shard sync tasks */ // NOTE: This has package level access solely for testing // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES @@ -517,13 +531,13 @@ public class Worker implements Runnable { IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization, Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool, WorkerStateChangeListener workerStateChangeListener, - LeaseCleanupValidator leaseCleanupValidator, LeaderDecider leaderDecider) { + LeaseCleanupValidator leaseCleanupValidator, LeaderDecider leaderDecider, PeriodicShardSyncManager periodicShardSyncManager) { this(applicationName, recordProcessorFactory, config, streamConfig, initialPositionInStream, parentShardPollIntervalMillis, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization, retryGetRecordsInSeconds, maxGetRecordsThreadPool, workerStateChangeListener, new KinesisShardSyncer(leaseCleanupValidator), - leaderDecider); + leaderDecider, periodicShardSyncManager); } Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, @@ -533,7 +547,8 @@ public class Worker implements Runnable { IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization, Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool, - WorkerStateChangeListener workerStateChangeListener, ShardSyncer shardSyncer, LeaderDecider leaderDecider) { + WorkerStateChangeListener workerStateChangeListener, ShardSyncer shardSyncer, LeaderDecider leaderDecider, + PeriodicShardSyncManager periodicShardSyncManager) { this.applicationName = applicationName; this.recordProcessorFactory = recordProcessorFactory; this.config = config; @@ -558,15 +573,17 @@ public class Worker implements Runnable { this.maxGetRecordsThreadPool = maxGetRecordsThreadPool; this.workerStateChangeListener = workerStateChangeListener; workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.CREATED); - this.leaderDecider = leaderDecider; this.shardSyncStrategy = createShardSyncStrategy(config.getShardSyncStrategyType()); LOG.info(String.format("Shard sync strategy determined as %s.", shardSyncStrategy.getStrategyType().toString())); + this.leaderDecider = leaderDecider != null ? leaderDecider : createLeaderDecider(); + this.leaderElectedPeriodicShardSyncManager = periodicShardSyncManager != null ? periodicShardSyncManager + : createPeriodicShardSyncManager(); } private ShardSyncStrategy createShardSyncStrategy(ShardSyncStrategyType strategyType) { switch (strategyType) { case PERIODIC: - return createPeriodicShardSyncStrategy(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager()); + return createPeriodicShardSyncStrategy(); case SHARD_END: default: return createShardEndShardSyncStrategy(controlServer); @@ -673,30 +690,30 @@ public class Worker implements Runnable { LOG.info("Initializing LeaseCoordinator"); leaseCoordinator.initialize(); - TaskResult result = null; - if (!skipShardSyncAtWorkerInitializationIfLeasesExist - || leaseCoordinator.getLeaseManager().isLeaseTableEmpty()) { - LOG.info("Syncing Kinesis shard info"); - ShardSyncTask shardSyncTask = new ShardSyncTask(streamConfig.getStreamProxy(), - leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion, - config.shouldIgnoreUnexpectedChildShards(), 0L, shardSyncer, null); - result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); - } else { - LOG.info("Skipping shard sync per config setting (and lease table is not empty)"); + // Perform initial lease sync if configs allow it, with jitter. + if (shouldInitiateLeaseSync()) { + LOG.info(config.getWorkerIdentifier() + " worker is beginning initial lease sync."); + TaskResult result = leaderElectedPeriodicShardSyncManager.syncShardsOnce(); + if (result.getException() != null) { + throw result.getException(); + } } - if (result == null || result.getException() == null) { - if (!leaseCoordinator.isRunning()) { - LOG.info("Starting LeaseCoordinator"); - leaseCoordinator.start(); - } else { - LOG.info("LeaseCoordinator is already running. No need to start it."); - } - shardSyncStrategy.onWorkerInitialization(); - isDone = true; + // If we reach this point, then we either skipped the lease sync or did not have any exception for the + // shard sync in the previous attempt. + if (!leaseCoordinator.isRunning()) { + LOG.info("Starting LeaseCoordinator"); + leaseCoordinator.start(); } else { - lastException = result.getException(); + LOG.info("LeaseCoordinator is already running. No need to start it."); } + + // All shard sync strategies' initialization handlers should begin a periodic shard sync. For + // PeriodicShardSync strategy, this is the main shard sync loop. For ShardEndShardSync and other + // shard sync strategies, this serves as an auditor background process. + shardSyncStrategy.onWorkerInitialization(); + isDone = true; + } catch (LeasingException e) { LOG.error("Caught exception when initializing LeaseCoordinator", e); lastException = e; @@ -706,6 +723,7 @@ public class Worker implements Runnable { try { Thread.sleep(parentShardPollIntervalMillis); + leaderElectedPeriodicShardSyncManager.stop(); } catch (InterruptedException e) { LOG.debug("Sleep interrupted while initializing worker."); } @@ -717,6 +735,32 @@ public class Worker implements Runnable { workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.STARTED); } + @VisibleForTesting + boolean shouldInitiateLeaseSync() throws InterruptedException, DependencyException, InvalidStateException, + ProvisionedThroughputException { + + final ILeaseManager leaseManager = leaseCoordinator.getLeaseManager(); + if (skipShardSyncAtWorkerInitializationIfLeasesExist && !leaseManager.isLeaseTableEmpty()) { + LOG.info("Skipping shard sync because getSkipShardSyncAtWorkerInitializationIfLeasesExist config is set " + + "to TRUE and lease table is not empty."); + return false; + } + + final long waitTime = ThreadLocalRandom.current().nextLong(MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS, + MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + final long waitUntil = System.currentTimeMillis() + waitTime; + + boolean shouldInitiateLeaseSync = true; + while (System.currentTimeMillis() < waitUntil && (shouldInitiateLeaseSync = leaseManager.isLeaseTableEmpty())) { + // Check every 3 seconds if lease table is still empty, to minimize contention between all workers + // bootstrapping from empty lease table at the same time. + LOG.info("Lease table is still empty. Checking again in " + LEASE_TABLE_CHECK_FREQUENCY_MILLIS + " ms."); + Thread.sleep(LEASE_TABLE_CHECK_FREQUENCY_MILLIS); + } + + return shouldInitiateLeaseSync; + } + /** * NOTE: This method is internal/private to the Worker class. It has package access solely for testing. * @@ -1163,20 +1207,33 @@ public class Worker implements Runnable { } } - private PeriodicShardSyncStrategy createPeriodicShardSyncStrategy(IKinesisProxy kinesisProxy, - ILeaseManager leaseManager) { - return new PeriodicShardSyncStrategy( - new PeriodicShardSyncManager(config.getWorkerIdentifier(), leaderDecider, - new ShardSyncTask(kinesisProxy, leaseManager, config.getInitialPositionInStreamExtended(), - config.shouldCleanupLeasesUponShardCompletion(), - config.shouldIgnoreUnexpectedChildShards(), SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, - shardSyncer, null), metricsFactory)); + private PeriodicShardSyncStrategy createPeriodicShardSyncStrategy() { + return new PeriodicShardSyncStrategy(createPeriodicShardSyncManager()); } private ShardEndShardSyncStrategy createShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager) { return new ShardEndShardSyncStrategy(shardSyncTaskManager); } + private LeaderDecider createLeaderDecider() { + return new DeterministicShuffleShardSyncLeaderDecider(leaseCoordinator.getLeaseManager(), + Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); + } + + private PeriodicShardSyncManager createPeriodicShardSyncManager() { + return new PeriodicShardSyncManager(config.getWorkerIdentifier(), + leaderDecider, + new ShardSyncTask(streamConfig.getStreamProxy(), + leaseCoordinator.getLeaseManager(), + config.getInitialPositionInStreamExtended(), + config.shouldCleanupLeasesUponShardCompletion(), + config.shouldIgnoreUnexpectedChildShards(), + SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, + shardSyncer, + null), + metricsFactory); + } + /** * Extension to CWMetricsFactory, so worker can identify whether it owns the metrics factory instance or not. * Visible and non-final only for testing. @@ -1241,6 +1298,8 @@ public class Worker implements Runnable { private ILeaseRenewer leaseRenewer; @Setter @Accessors(fluent = true) private ShardSyncer shardSyncer; + @Setter @Accessors(fluent = true) + private PeriodicShardSyncManager periodicShardSyncManager; @VisibleForTesting @@ -1379,7 +1438,7 @@ public class Worker implements Runnable { } // We expect users to either inject both LeaseRenewer and the corresponding thread-pool, or neither of them (DEFAULT). - if (leaseRenewer == null){ + if (leaseRenewer == null) { ExecutorService leaseRenewerThreadPool = LeaseCoordinator.getDefaultLeaseRenewalExecutorService(config.getMaxLeaseRenewalThreads()); leaseRenewer = new LeaseRenewer<>(leaseManager, config.getWorkerIdentifier(), config.getFailoverTimeMillis(), leaseRenewerThreadPool); } @@ -1389,6 +1448,20 @@ public class Worker implements Runnable { Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); } + if (periodicShardSyncManager == null) { + periodicShardSyncManager = new PeriodicShardSyncManager(config.getWorkerIdentifier(), + leaderDecider, + new ShardSyncTask(kinesisProxy, + leaseManager, + config.getInitialPositionInStreamExtended(), + config.shouldCleanupLeasesUponShardCompletion(), + config.shouldIgnoreUnexpectedChildShards(), + SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, + shardSyncer, + null), + metricsFactory); + } + return new Worker(config.getApplicationName(), recordProcessorFactory, config, @@ -1419,7 +1492,10 @@ public class Worker implements Runnable { shardPrioritization, config.getRetryGetRecordsInSeconds(), config.getMaxGetRecordsThreadPool(), - workerStateChangeListener, shardSyncer, leaderDecider); + workerStateChangeListener, + shardSyncer, + leaderDecider, + periodicShardSyncManager); } > R createClient(final T builder, diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index 83dcd4af..48d71f6d 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -18,7 +18,6 @@ import java.io.File; import java.io.IOException; import java.math.BigInteger; import java.util.ArrayList; -import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -30,7 +29,6 @@ import java.util.stream.Stream; import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; import com.amazonaws.services.dynamodbv2.local.embedded.DynamoDBEmbedded; -import com.amazonaws.services.kinesis.leases.impl.Lease; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; import org.apache.commons.logging.Log; @@ -60,7 +58,7 @@ import com.amazonaws.services.kinesis.model.Shard; import junit.framework.Assert; -import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; @@ -1816,10 +1814,13 @@ public class ShardSyncerTest { dataFile.deleteOnExit(); final IKinesisProxy kinesisProxy = spy(new KinesisLocalFileProxy(dataFile.getAbsolutePath())); + // Make sure ListShardsWithFilter is called in all public shard sync methods shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, false); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPosition, + cleanupLeasesOfCompletedShards, false, null); - verify(kinesisProxy, atLeastOnce()).getShardListWithFilter(shardFilter); + verify(kinesisProxy, atLeast(2)).getShardListWithFilter(shardFilter); verify(kinesisProxy, never()).getShardList(); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index a2faf607..b4b2a597 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -72,6 +72,7 @@ import com.amazonaws.services.kinesis.leases.impl.KinesisClientLeaseBuilder; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLeaseManager; import com.amazonaws.services.kinesis.leases.impl.LeaseManager; import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsScope; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.hamcrest.Condition; @@ -158,6 +159,7 @@ public class WorkerTest { private static final String KINESIS_SHARD_ID_FORMAT = "kinesis-0-0-%d"; private static final String CONCURRENCY_TOKEN_FORMAT = "testToken-%d"; + private static final String WORKER_ID = "workerId"; private RecordsFetcherFactory recordsFetcherFactory; private KinesisClientLibConfiguration config; @@ -194,7 +196,7 @@ public class WorkerTest { @Before public void setup() { - config = spy(new KinesisClientLibConfiguration("app", null, null, null)); + config = spy(new KinesisClientLibConfiguration("app", null, null, WORKER_ID)); recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); } @@ -244,7 +246,7 @@ public class WorkerTest { @Test public final void testGetStageName() { final String stageName = "testStageName"; - config = new KinesisClientLibConfiguration(stageName, null, null, null); + config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); Worker worker = new Worker(v1RecordProcessorFactory, config); Assert.assertEquals(stageName, worker.getApplicationName()); } @@ -253,7 +255,7 @@ public class WorkerTest { public final void testCreateOrGetShardConsumer() { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; - config = new KinesisClientLibConfiguration(stageName, null, null, null); + config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; @@ -372,7 +374,7 @@ public class WorkerTest { public final void testCleanupShardConsumers() { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; - config = new KinesisClientLibConfiguration(stageName, null, null, null); + config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; @@ -429,12 +431,12 @@ public class WorkerTest { } @Test - public final void testInitializationFailureWithRetries() { + public final void testInitializationFailureWithRetries() throws Exception { String stageName = "testInitializationWorker"; IRecordProcessorFactory recordProcessorFactory = new TestStreamletFactory(null, null); - config = new KinesisClientLibConfiguration(stageName, null, null, null); + config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); int count = 0; - when(proxy.getShardList()).thenThrow(new RuntimeException(Integer.toString(count++))); + when(proxy.getShardListWithFilter(any())).thenThrow(new RuntimeException(Integer.toString(count++))); int maxRecords = 2; long idleTimeInMilliseconds = 1L; StreamConfig streamConfig = @@ -443,6 +445,7 @@ public class WorkerTest { idleTimeInMilliseconds, callProcessRecordsForEmptyRecordList, skipCheckpointValidationValue, INITIAL_POSITION_LATEST); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + when(leaseManager.isLeaseTableEmpty()).thenReturn(true); ExecutorService execService = Executors.newSingleThreadExecutor(); long shardPollInterval = 0L; Worker worker = @@ -465,6 +468,79 @@ public class WorkerTest { Assert.assertTrue(count > 0); } + @Test + public final void testInitializationWaitsWhenLeaseTableIsEmpty() throws Exception { + final String stageName = "testInitializationWorker"; + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + when(leaseManager.isLeaseTableEmpty()).thenReturn(true); + + final int maxRecords = 2; + final long idleTimeInMilliseconds = 1L; + final StreamConfig streamConfig = new StreamConfig(proxy, maxRecords, idleTimeInMilliseconds, + callProcessRecordsForEmptyRecordList, skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + final long shardPollInterval = 0L; + final Worker worker = + new Worker(stageName, + v2RecordProcessorFactory, + config, + streamConfig, INITIAL_POSITION_TRIM_HORIZON, + shardPollInterval, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + Executors.newSingleThreadExecutor(), + nullMetricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + shardPrioritization); + + final long startTime = System.currentTimeMillis(); + worker.shouldInitiateLeaseSync(); + final long endTime = System.currentTimeMillis(); + + assertTrue(endTime - startTime > Worker.MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + assertTrue(endTime - startTime < Worker.MAX_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS + Worker.LEASE_TABLE_CHECK_FREQUENCY_MILLIS); + } + + @Test + public final void testInitializationDoesntWaitWhenLeaseTableIsNotEmpty() throws Exception { + final String stageName = "testInitializationWorker"; + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + when(leaseManager.isLeaseTableEmpty()).thenReturn(false); + + final int maxRecords = 2; + final long idleTimeInMilliseconds = 1L; + final StreamConfig streamConfig = new StreamConfig(proxy, maxRecords, idleTimeInMilliseconds, + callProcessRecordsForEmptyRecordList, skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + final long shardPollInterval = 0L; + final Worker worker = + new Worker(stageName, + v2RecordProcessorFactory, + config, + streamConfig, INITIAL_POSITION_TRIM_HORIZON, + shardPollInterval, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + Executors.newSingleThreadExecutor(), + nullMetricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + shardPrioritization); + + final long startTime = System.currentTimeMillis(); + worker.shouldInitiateLeaseSync(); + final long endTime = System.currentTimeMillis(); + + assertTrue(endTime - startTime < Worker.MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS); + } + /** * Runs worker with threadPoolSize == numShards * Test method for {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker#run()}. @@ -576,6 +652,7 @@ public class WorkerTest { final ExecutorService executorService = mock(ThreadPoolExecutor.class); final CWMetricsFactory cwMetricsFactory = mock(CWMetricsFactory.class); + when(cwMetricsFactory.createMetrics()).thenReturn(mock(IMetricsScope.class)); // Make sure that worker thread is run before invoking shutdown. final CountDownLatch workerStarted = new CountDownLatch(1); doAnswer(new Answer() { @@ -1708,7 +1785,7 @@ public class WorkerTest { public void testBuilderSetRegionAndEndpointToClient() { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); final String endpoint = "TestEndpoint"; - KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, null) + KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, WORKER_ID) .withRegionName(Regions.US_WEST_2.getName()) .withKinesisEndpoint(endpoint) .withDynamoDBEndpoint(endpoint); @@ -1736,7 +1813,7 @@ public class WorkerTest { public void testBuilderSetRegionToClient() { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); String region = Regions.US_WEST_2.getName(); - KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, null) + KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, WORKER_ID) .withRegionName(region); Worker.Builder builder = new Worker.Builder(); @@ -1763,7 +1840,7 @@ public class WorkerTest { @Test public void testBuilderGenerateClients() { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, null); + KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, WORKER_ID); Worker.Builder builder = spy(new Worker.Builder().recordProcessorFactory(recordProcessorFactory).config(config)); ArgumentCaptor builderCaptor = ArgumentCaptor.forClass(AwsClientBuilder.class); @@ -1789,7 +1866,7 @@ public class WorkerTest { public void testBuilderGenerateClientsWithRegion() { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); String region = Regions.US_WEST_2.getName(); - KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, null) + KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, WORKER_ID) .withRegionName(region); ArgumentCaptor builderCaptor = ArgumentCaptor.forClass(AwsClientBuilder.class); @@ -1809,7 +1886,7 @@ public class WorkerTest { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); String region = Regions.US_WEST_2.getName(); String endpointUrl = "TestEndpoint"; - KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, null) + KinesisClientLibConfiguration config = new KinesisClientLibConfiguration("TestApp", null, null, WORKER_ID) .withRegionName(region).withKinesisEndpoint(endpointUrl).withDynamoDBEndpoint(endpointUrl); Worker.Builder builder = spy(new Worker.Builder()); From fdad20aff469e8cef9a42f81836a793ad88d1cbc Mon Sep 17 00:00:00 2001 From: Micah Jaffe <31011877+micah-jaffe@users.noreply.github.com> Date: Thu, 14 May 2020 09:04:44 -0700 Subject: [PATCH 04/43] Add leader-elected periodic shard sync manager as auditor to ShardEndShardSync strategy (#35) * Add leader-elected periodic shard sync manager as auditor to ShardEndShardSync strategy * Address PR feedback and add Builder unit tests --- .../lib/worker/ShardEndShardSyncStrategy.java | 16 ++-- .../clientlibrary/lib/worker/Worker.java | 92 ++++++++++++------- .../clientlibrary/lib/worker/WorkerTest.java | 58 ++++++++++++ 3 files changed, 126 insertions(+), 40 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java index 18fc4a08..9efe2f51 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardEndShardSyncStrategy.java @@ -16,8 +16,13 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy { private static final Log LOG = LogFactory.getLog(Worker.class); private ShardSyncTaskManager shardSyncTaskManager; - ShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager) { + /** Runs periodic shard sync jobs in the background as an auditor process for shard-end syncs. */ + private PeriodicShardSyncManager periodicShardSyncManager; + + ShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager, + PeriodicShardSyncManager periodicShardSyncManager) { this.shardSyncTaskManager = shardSyncTaskManager; + this.periodicShardSyncManager = periodicShardSyncManager; } @Override @@ -42,9 +47,8 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy { @Override public TaskResult onWorkerInitialization() { - // TODO: Start leaderElectedPeriodicShardSyncManager in background - LOG.debug(String.format("onWorkerInitialization is NoOp for ShardSyncStrategyType %s", getStrategyType().toString())); - return new TaskResult(null); + LOG.info("Starting periodic shard sync background process for SHARD_END shard sync strategy."); + return periodicShardSyncManager.start(); } @Override @@ -66,7 +70,7 @@ class ShardEndShardSyncStrategy implements ShardSyncStrategy { @Override public void onWorkerShutDown() { - // TODO: Shut down leaderElectedPeriodicShardSyncManager - LOG.debug(String.format("Stop is NoOp for ShardSyncStrategyType %s", getStrategyType().toString())); + LOG.info("Stopping periodic shard sync background process for SHARD_END shard sync strategy."); + periodicShardSyncManager.stop(); } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 5f1b4fac..eec3910d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -126,7 +126,7 @@ public class Worker implements Runnable { private final Optional maxGetRecordsThreadPool; private final KinesisClientLibLeaseCoordinator leaseCoordinator; - private final ShardSyncTaskManager controlServer; + private final ShardSyncTaskManager shardSyncTaskManager; private final ShardPrioritization shardPrioritization; @@ -562,7 +562,7 @@ public class Worker implements Runnable { this.leaseCoordinator = leaseCoordinator; this.metricsFactory = metricsFactory; this.shardSyncer = shardSyncer; - this.controlServer = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(), + this.shardSyncTaskManager = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(), initialPositionInStream, cleanupLeasesUponShardCompletion, config.shouldIgnoreUnexpectedChildShards(), shardSyncIdleTimeMillis, metricsFactory, executorService, shardSyncer); this.taskBackoffTimeMillis = taskBackoffTimeMillis; @@ -573,21 +573,37 @@ public class Worker implements Runnable { this.maxGetRecordsThreadPool = maxGetRecordsThreadPool; this.workerStateChangeListener = workerStateChangeListener; workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.CREATED); - this.shardSyncStrategy = createShardSyncStrategy(config.getShardSyncStrategyType()); - LOG.info(String.format("Shard sync strategy determined as %s.", shardSyncStrategy.getStrategyType().toString())); - this.leaderDecider = leaderDecider != null ? leaderDecider : createLeaderDecider(); - this.leaderElectedPeriodicShardSyncManager = periodicShardSyncManager != null ? periodicShardSyncManager - : createPeriodicShardSyncManager(); + createShardSyncStrategy(config.getShardSyncStrategyType(), leaderDecider, periodicShardSyncManager); } - private ShardSyncStrategy createShardSyncStrategy(ShardSyncStrategyType strategyType) { + /** + * Create shard sync strategy and corresponding {@link LeaderDecider} based on provided configs. PERIODIC + * {@link ShardSyncStrategyType} honors custom leaderDeciders for leader election strategy. All other + * {@link ShardSyncStrategyType}s permit only a default single-leader strategy. + */ + private void createShardSyncStrategy(ShardSyncStrategyType strategyType, + LeaderDecider leaderDecider, + PeriodicShardSyncManager periodicShardSyncManager) { switch (strategyType) { case PERIODIC: - return createPeriodicShardSyncStrategy(); + this.leaderDecider = getOrCreateLeaderDecider(leaderDecider); + this.leaderElectedPeriodicShardSyncManager = + getOrCreatePeriodicShardSyncManager(periodicShardSyncManager); + this.shardSyncStrategy = createPeriodicShardSyncStrategy(); + break; case SHARD_END: default: - return createShardEndShardSyncStrategy(controlServer); + if (leaderDecider != null) { + LOG.warn("LeaderDecider cannot be customized with non-PERIODIC shard sync strategy type. Using " + + "default LeaderDecider."); + } + this.leaderDecider = getOrCreateLeaderDecider(null); + this.leaderElectedPeriodicShardSyncManager = + getOrCreatePeriodicShardSyncManager(periodicShardSyncManager); + this.shardSyncStrategy = createShardEndShardSyncStrategy(); } + + LOG.info("Shard sync strategy determined as " + shardSyncStrategy.getStrategyType().toString()); } private static KinesisClientLibLeaseCoordinator getLeaseCoordinator(KinesisClientLibConfiguration config, @@ -619,6 +635,20 @@ public class Worker implements Runnable { return leaseCoordinator; } + /** + * @return the leaderDecider + */ + LeaderDecider getLeaderDecider() { + return leaderDecider; + } + + /** + * @return the leaderElectedPeriodicShardSyncManager + */ + PeriodicShardSyncManager getPeriodicShardSyncManager() { + return leaderElectedPeriodicShardSyncManager; + } + /** * Start consuming data from the stream, and pass it to the application record processors. */ @@ -723,14 +753,15 @@ public class Worker implements Runnable { try { Thread.sleep(parentShardPollIntervalMillis); - leaderElectedPeriodicShardSyncManager.stop(); } catch (InterruptedException e) { LOG.debug("Sleep interrupted while initializing worker."); } } if (!isDone) { + leaderElectedPeriodicShardSyncManager.stop(); throw new RuntimeException(lastException); + } workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.STARTED); } @@ -1208,19 +1239,29 @@ public class Worker implements Runnable { } private PeriodicShardSyncStrategy createPeriodicShardSyncStrategy() { - return new PeriodicShardSyncStrategy(createPeriodicShardSyncManager()); + return new PeriodicShardSyncStrategy(leaderElectedPeriodicShardSyncManager); } - private ShardEndShardSyncStrategy createShardEndShardSyncStrategy(ShardSyncTaskManager shardSyncTaskManager) { - return new ShardEndShardSyncStrategy(shardSyncTaskManager); + private ShardEndShardSyncStrategy createShardEndShardSyncStrategy() { + return new ShardEndShardSyncStrategy(shardSyncTaskManager, leaderElectedPeriodicShardSyncManager); } - private LeaderDecider createLeaderDecider() { + private LeaderDecider getOrCreateLeaderDecider(LeaderDecider leaderDecider) { + if (leaderDecider != null) { + return leaderDecider; + } + return new DeterministicShuffleShardSyncLeaderDecider(leaseCoordinator.getLeaseManager(), Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); } - private PeriodicShardSyncManager createPeriodicShardSyncManager() { + private PeriodicShardSyncManager getOrCreatePeriodicShardSyncManager(PeriodicShardSyncManager periodicShardSyncManager) { + // TODO: Configure periodicShardSyncManager with either mandatory shard sync (PERIODIC) or hash range + // validation based shard sync (SHARD_END) based on configured shard sync strategy + if (periodicShardSyncManager != null) { + return periodicShardSyncManager; + } + return new PeriodicShardSyncManager(config.getWorkerIdentifier(), leaderDecider, new ShardSyncTask(streamConfig.getStreamProxy(), @@ -1298,9 +1339,6 @@ public class Worker implements Runnable { private ILeaseRenewer leaseRenewer; @Setter @Accessors(fluent = true) private ShardSyncer shardSyncer; - @Setter @Accessors(fluent = true) - private PeriodicShardSyncManager periodicShardSyncManager; - @VisibleForTesting AmazonKinesis getKinesisClient() { @@ -1448,20 +1486,6 @@ public class Worker implements Runnable { Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); } - if (periodicShardSyncManager == null) { - periodicShardSyncManager = new PeriodicShardSyncManager(config.getWorkerIdentifier(), - leaderDecider, - new ShardSyncTask(kinesisProxy, - leaseManager, - config.getInitialPositionInStreamExtended(), - config.shouldCleanupLeasesUponShardCompletion(), - config.shouldIgnoreUnexpectedChildShards(), - SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, - shardSyncer, - null), - metricsFactory); - } - return new Worker(config.getApplicationName(), recordProcessorFactory, config, @@ -1495,7 +1519,7 @@ public class Worker implements Runnable { workerStateChangeListener, shardSyncer, leaderDecider, - periodicShardSyncManager); + null /* PeriodicShardSyncManager */); } > R createClient(final T builder, diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index b4b2a597..08ad6efd 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -1781,6 +1781,64 @@ public class WorkerTest { Assert.assertSame(leaseManager, worker.getLeaseCoordinator().getLeaseManager()); } + @Test + public void testBuilderWithDefaultShardSyncStrategy() { + IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); + Worker worker = new Worker.Builder() + .recordProcessorFactory(recordProcessorFactory) + .config(config) + .build(); + + Assert.assertNotNull(worker.getLeaderDecider()); + Assert.assertNotNull(worker.getPeriodicShardSyncManager()); + } + + @Test + public void testBuilderWithPeriodicShardSyncStrategyAndDefaultLeaderDecider() { + IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); + when(config.getShardSyncStrategyType()).thenReturn(ShardSyncStrategyType.PERIODIC); + Worker worker = new Worker.Builder() + .recordProcessorFactory(recordProcessorFactory) + .config(config) + .build(); + + Assert.assertNotNull(worker.getLeaderDecider()); + Assert.assertNotNull(worker.getPeriodicShardSyncManager()); + } + + @Test + public void testBuilderWithPeriodicShardSyncStrategyAndCustomLeaderDecider() { + IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); + when(config.getShardSyncStrategyType()).thenReturn(ShardSyncStrategyType.PERIODIC); + + LeaderDecider leaderDecider = mock(LeaderDecider.class); + Worker worker = new Worker.Builder() + .recordProcessorFactory(recordProcessorFactory) + .config(config) + .leaderDecider(leaderDecider) + .build(); + + Assert.assertSame(leaderDecider, worker.getLeaderDecider()); + Assert.assertNotNull(worker.getPeriodicShardSyncManager()); + } + + @Test + public void testCustomLeaderDeciderNotAllowedForShardEndShardSync() { + IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); + when(config.getShardSyncStrategyType()).thenReturn(ShardSyncStrategyType.SHARD_END); + + LeaderDecider leaderDecider = mock(LeaderDecider.class); + Worker worker = new Worker.Builder() + .recordProcessorFactory(recordProcessorFactory) + .config(config) + .leaderDecider(leaderDecider) + .build(); + + // Worker should override custom leaderDecider and use default instead + Assert.assertNotSame(leaderDecider, worker.getLeaderDecider()); + Assert.assertNotNull(worker.getPeriodicShardSyncManager()); + } + @Test public void testBuilderSetRegionAndEndpointToClient() { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); From 6b474b73908151108436d66790645737fe79d7e7 Mon Sep 17 00:00:00 2001 From: Micah Jaffe <31011877+micah-jaffe@users.noreply.github.com> Date: Fri, 15 May 2020 14:02:44 -0700 Subject: [PATCH 05/43] Add configurable max initialization attempts (#39) * Add configurable max initialization attempts * Add maxInitializationAttempts to unit test setup --- .../worker/KinesisClientLibConfiguration.java | 18 ++++++++++++++++++ .../clientlibrary/lib/worker/Worker.java | 6 +++--- .../clientlibrary/lib/worker/WorkerTest.java | 4 +++- 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index ba01bc9e..86e7a496 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -200,6 +200,11 @@ public class KinesisClientLibConfiguration { */ public static final int DEFAULT_MAX_LIST_SHARDS_RETRY_ATTEMPTS = 50; + /** + * The number of times the {@link Worker} will try to initialize before giving up. + */ + public static final int DEFAULT_MAX_INITIALIZATION_ATTEMPTS = 20; + @Getter private BillingMode billingMode; private String applicationName; @@ -266,6 +271,9 @@ public class KinesisClientLibConfiguration { @Getter private int maxListShardsRetryAttempts = DEFAULT_MAX_LIST_SHARDS_RETRY_ATTEMPTS; + @Getter + private int maxInitializationAttempts = DEFAULT_MAX_INITIALIZATION_ATTEMPTS; + /** * Constructor. * @@ -1458,4 +1466,14 @@ public class KinesisClientLibConfiguration { this.maxListShardsRetryAttempts = maxListShardsRetryAttempts; return this; } + + /** + * @param maxInitializationAttempts Max number of Worker initialization attempts before giving up + * @return + */ + public KinesisClientLibConfiguration withMaxInitializationAttempts(int maxInitializationAttempts) { + checkIsValuePositive("maxInitializationAttempts", maxInitializationAttempts); + this.maxInitializationAttempts = maxInitializationAttempts; + return this; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index eec3910d..a86771e3 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -94,7 +94,6 @@ public class Worker implements Runnable { // Default configs for periodic shard sync private static final int SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC = 0; - private static final int MAX_INITIALIZATION_ATTEMPTS = 20; private static final int PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT = 1; //Default for KCL. static final long LEASE_TABLE_CHECK_FREQUENCY_MILLIS = 3 * 1000L; static final long MIN_WAIT_TIME_FOR_LEASE_TABLE_CHECK_MILLIS = 1 * 1000L; @@ -661,7 +660,8 @@ public class Worker implements Runnable { initialize(); LOG.info("Initialization complete. Starting worker loop."); } catch (RuntimeException e1) { - LOG.error("Unable to initialize after " + MAX_INITIALIZATION_ATTEMPTS + " attempts. Shutting down.", e1); + LOG.error("Unable to initialize after " + config.getMaxInitializationAttempts() + " attempts. " + + "Shutting down.", e1); shutdown(); } @@ -714,7 +714,7 @@ public class Worker implements Runnable { boolean isDone = false; Exception lastException = null; - for (int i = 0; (!isDone) && (i < MAX_INITIALIZATION_ATTEMPTS); i++) { + for (int i = 0; (!isDone) && (i < config.getMaxInitializationAttempts()); i++) { try { LOG.info("Initialization attempt " + (i + 1)); LOG.info("Initializing LeaseCoordinator"); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index 08ad6efd..f6bfdd73 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -197,6 +197,7 @@ public class WorkerTest { @Before public void setup() { config = spy(new KinesisClientLibConfiguration("app", null, null, WORKER_ID)); + config.withMaxInitializationAttempts(1); recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); } @@ -435,6 +436,8 @@ public class WorkerTest { String stageName = "testInitializationWorker"; IRecordProcessorFactory recordProcessorFactory = new TestStreamletFactory(null, null); config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); + config.withMaxInitializationAttempts(2); + int count = 0; when(proxy.getShardListWithFilter(any())).thenThrow(new RuntimeException(Integer.toString(count++))); int maxRecords = 2; @@ -612,7 +615,6 @@ public class WorkerTest { @Test public final void testWorkerShutsDownOwnedResources() throws Exception { - final long failoverTimeMillis = 20L; // Make sure that worker thread is run before invoking shutdown. From 3a88a60a4ee7bb8f969b36cb79e7665a7395b6ec Mon Sep 17 00:00:00 2001 From: Micah Jaffe <31011877+micah-jaffe@users.noreply.github.com> Date: Mon, 22 Jun 2020 08:45:46 -0700 Subject: [PATCH 06/43] Add periodic hash range auditor, hole detection, and recovery (#43) * Add periodic hash range auditor, hole detection, and recovery * Add unit tests for hash range hole recovery * Fix max hash range bug * Address PR feedback * Fix DDB hash key persistence bug --- .../lib/worker/PeriodicShardSyncManager.java | 305 ++++++++- .../clientlibrary/lib/worker/Worker.java | 21 +- .../types/ExtendedSequenceNumber.java | 4 + .../leases/impl/HashKeyRangeForLease.java | 79 +++ .../leases/impl/KinesisClientLease.java | 46 +- .../impl/KinesisClientLeaseSerializer.java | 29 + .../kinesis/leases/impl/LeaseManager.java | 29 + .../kinesis/leases/impl/LeaseSerializer.java | 6 + .../kinesis/leases/impl/UpdateField.java | 26 + .../leases/interfaces/ILeaseManager.java | 14 + .../leases/interfaces/ILeaseSerializer.java | 10 + .../worker/ExceptionThrowingLeaseManager.java | 10 + .../worker/PeriodicShardSyncManagerTest.java | 613 ++++++++++++++++++ .../clientlibrary/lib/worker/WorkerTest.java | 16 +- .../impl/KinesisClientLeaseBuilder.java | 15 +- 15 files changed, 1191 insertions(+), 32 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/leases/impl/HashKeyRangeForLease.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/leases/impl/UpdateField.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java index 87237e0b..75a747e6 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java @@ -14,39 +14,98 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import java.io.Serializable; +import java.math.BigInteger; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease; +import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; +import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.util.CollectionUtils; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ComparisonChain; import lombok.EqualsAndHashCode; import lombok.Getter; +import lombok.NonNull; +import lombok.Value; +import lombok.experimental.Accessors; import org.apache.commons.lang3.Validate; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.fromHashKeyRange; + /** - * The top level orchestrator for coordinating the periodic shard sync related - * activities. + * The top level orchestrator for coordinating the periodic shard sync related activities. If the configured + * {@link ShardSyncStrategyType} is PERIODIC, this class will be the main shard sync orchestrator. For non-PERIODIC + * strategies, this class will serve as an internal auditor that periodically checks if the full hash range is covered + * by currently held leases, and initiates a recovery shard sync if not. */ @Getter @EqualsAndHashCode class PeriodicShardSyncManager { private static final Log LOG = LogFactory.getLog(PeriodicShardSyncManager.class); private static final long INITIAL_DELAY = 0; - private static final long PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000; + + /** DEFAULT interval is used for PERIODIC {@link ShardSyncStrategyType}. */ + private static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000L; + + /** AUDITOR interval is used for non-PERIODIC {@link ShardSyncStrategyType} auditor processes. */ + private static final long AUDITOR_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L; + + /** Parameters for validating hash range completeness when running in auditor mode. */ + @VisibleForTesting + static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; + @VisibleForTesting + static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); + @VisibleForTesting + static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3; + private final HashRangeHoleTracker hashRangeHoleTracker = new HashRangeHoleTracker(); private final String workerId; private final LeaderDecider leaderDecider; private final ITask metricsEmittingShardSyncTask; private final ScheduledExecutorService shardSyncThreadPool; + private final ILeaseManager leaseManager; + private final IKinesisProxy kinesisProxy; + private final boolean isAuditorMode; + private final long periodicShardSyncIntervalMillis; private boolean isRunning; - PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, IMetricsFactory metricsFactory) { - this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory); + PeriodicShardSyncManager(String workerId, + LeaderDecider leaderDecider, + ShardSyncTask shardSyncTask, + IMetricsFactory metricsFactory, + ILeaseManager leaseManager, + IKinesisProxy kinesisProxy, + boolean isAuditorMode) { + this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory, + leaseManager, kinesisProxy, isAuditorMode); } - PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, ShardSyncTask shardSyncTask, ScheduledExecutorService shardSyncThreadPool, IMetricsFactory metricsFactory) { + PeriodicShardSyncManager(String workerId, + LeaderDecider leaderDecider, + ShardSyncTask shardSyncTask, + ScheduledExecutorService shardSyncThreadPool, + IMetricsFactory metricsFactory, + ILeaseManager leaseManager, + IKinesisProxy kinesisProxy, + boolean isAuditorMode) { Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager."); @@ -54,6 +113,16 @@ class PeriodicShardSyncManager { this.leaderDecider = leaderDecider; this.metricsEmittingShardSyncTask = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory); this.shardSyncThreadPool = shardSyncThreadPool; + this.leaseManager = leaseManager; + this.kinesisProxy = kinesisProxy; + this.isAuditorMode = isAuditorMode; + if (isAuditorMode) { + Validate.notNull(this.leaseManager, "LeaseManager is required for non-PERIODIC shard sync strategies."); + Validate.notNull(this.kinesisProxy, "KinesisProxy is required for non-PERIODIC shard sync strategies."); + this.periodicShardSyncIntervalMillis = AUDITOR_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS; + } else { + this.periodicShardSyncIntervalMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS; + } } public synchronized TaskResult start() { @@ -67,7 +136,7 @@ class PeriodicShardSyncManager { }; shardSyncThreadPool - .scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, PERIODIC_SHARD_SYNC_INTERVAL_MILLIS, + .scheduleWithFixedDelay(periodicShardSyncer, INITIAL_DELAY, periodicShardSyncIntervalMillis, TimeUnit.MILLISECONDS); isRunning = true; } @@ -95,11 +164,227 @@ class PeriodicShardSyncManager { private void runShardSync() { if (leaderDecider.isLeader(workerId)) { - LOG.debug(String.format("WorkerId %s is a leader, running the shard sync task", workerId)); - metricsEmittingShardSyncTask.call(); + LOG.debug("WorkerId " + workerId + " is a leader, running the shard sync task"); + + try { + final ShardSyncResponse shardSyncResponse = checkForShardSync(); + if (shardSyncResponse.shouldDoShardSync()) { + LOG.info("Periodic shard syncer initiating shard sync due to the reason - " + + shardSyncResponse.reasonForDecision()); + metricsEmittingShardSyncTask.call(); + } else { + LOG.info("Skipping shard sync due to the reason - " + shardSyncResponse.reasonForDecision()); + } + } 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 " + workerId + " is not a leader, not running the shard sync task"); + } + } + + @VisibleForTesting + ShardSyncResponse checkForShardSync() throws DependencyException, InvalidStateException, + ProvisionedThroughputException { + + if (!isAuditorMode) { + // If we are running with PERIODIC shard sync strategy, we should sync every time. + return new ShardSyncResponse(true, "Syncing every time with PERIODIC shard sync strategy."); } + // Get current leases from DynamoDB. + final List currentLeases = leaseManager.listLeases(); + + if (CollectionUtils.isNullOrEmpty(currentLeases)) { + // If the current leases are null or empty, then we need to initiate a shard sync. + LOG.info("No leases found. Will trigger a shard sync."); + return new ShardSyncResponse(true, "No leases found."); + } + + // Check if there are any holes in the hash range covered by current leases. Return the first hole if present. + Optional hashRangeHoleOpt = hasHoleInLeases(currentLeases); + if (hashRangeHoleOpt.isPresent()) { + // If hole is present, check if the hole is detected consecutively in previous occurrences. If hole is + // determined with high confidence, return true; return false otherwise. We use the high confidence factor + // to avoid shard sync on any holes during resharding and lease cleanups, or other intermittent issues. + final boolean hasHoleWithHighConfidence = + hashRangeHoleTracker.hashHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); + + return new ShardSyncResponse(hasHoleWithHighConfidence, + "Detected the same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + " times. " + + "Will initiate shard sync after reaching threshold: " + CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY); + } else { + // If hole is not present, clear any previous hole tracking and return false. + hashRangeHoleTracker.reset(); + return new ShardSyncResponse(false, "Hash range is complete."); + } + } + + @VisibleForTesting + Optional hasHoleInLeases(List leases) { + // Filter out any leases with checkpoints other than SHARD_END + final List activeLeases = leases.stream() + .filter(lease -> lease.getCheckpoint() != null && !lease.getCheckpoint().isShardEnd()) + .collect(Collectors.toList()); + + final List activeLeasesWithHashRanges = fillWithHashRangesIfRequired(activeLeases); + return checkForHoleInHashKeyRanges(activeLeasesWithHashRanges); + } + + private List fillWithHashRangesIfRequired(List activeLeases) { + final List activeLeasesWithNoHashRanges = activeLeases.stream() + .filter(lease -> lease.getHashKeyRange() == null).collect(Collectors.toList()); + + if (activeLeasesWithNoHashRanges.isEmpty()) { + return activeLeases; + } + + // Fetch shards from Kinesis to fill in the in-memory hash ranges + final Map kinesisShards = kinesisProxy.getShardList().stream() + .collect(Collectors.toMap(Shard::getShardId, shard -> shard)); + + return activeLeases.stream().map(lease -> { + if (lease.getHashKeyRange() == null) { + final String shardId = lease.getLeaseKey(); + final Shard shard = kinesisShards.get(shardId); + if (shard == null) { + return lease; + } + lease.setHashKeyRange(fromHashKeyRange(shard.getHashKeyRange())); + + try { + leaseManager.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + } catch (Exception e) { + LOG.warn("Unable to update hash range information for lease " + lease.getLeaseKey() + + ". This may result in explicit lease sync."); + } + } + return lease; + }).filter(lease -> lease.getHashKeyRange() != null).collect(Collectors.toList()); + } + + @VisibleForTesting + static Optional checkForHoleInHashKeyRanges(List leasesWithHashKeyRanges) { + // Sort the hash ranges by starting hash key + final List sortedLeasesWithHashKeyRanges = sortLeasesByHashRange(leasesWithHashKeyRanges); + if (sortedLeasesWithHashKeyRanges.isEmpty()) { + LOG.error("No leases with valid hash ranges found."); + return Optional.of(new HashRangeHole()); + } + + // Validate the hash range bounds + final KinesisClientLease minHashKeyLease = sortedLeasesWithHashKeyRanges.get(0); + final KinesisClientLease maxHashKeyLease = + sortedLeasesWithHashKeyRanges.get(sortedLeasesWithHashKeyRanges.size() - 1); + if (!minHashKeyLease.getHashKeyRange().startingHashKey().equals(MIN_HASH_KEY) || + !maxHashKeyLease.getHashKeyRange().endingHashKey().equals(MAX_HASH_KEY)) { + LOG.error("Incomplete hash range found between " + minHashKeyLease + " and " + maxHashKeyLease); + return Optional.of(new HashRangeHole(minHashKeyLease.getHashKeyRange(), maxHashKeyLease.getHashKeyRange())); + } + + // Check for any holes in the sorted hash range intervals + if (sortedLeasesWithHashKeyRanges.size() > 1) { + KinesisClientLease leftmostLeaseToReportInCaseOfHole = minHashKeyLease; + HashKeyRangeForLease leftLeaseHashRange = leftmostLeaseToReportInCaseOfHole.getHashKeyRange(); + + for (int i = 1; i < sortedLeasesWithHashKeyRanges.size(); i++) { + final KinesisClientLease rightLease = sortedLeasesWithHashKeyRanges.get(i); + final HashKeyRangeForLease rightLeaseHashRange = rightLease.getHashKeyRange(); + final BigInteger rangeDiff = + rightLeaseHashRange.startingHashKey().subtract(leftLeaseHashRange.endingHashKey()); + // We have overlapping leases when rangeDiff is 0 or negative. + // signum() will be -1 for negative and 0 if value is 0. + // Merge the ranges for further tracking. + if (rangeDiff.signum() <= 0) { + leftLeaseHashRange = new HashKeyRangeForLease(leftLeaseHashRange.startingHashKey(), + leftLeaseHashRange.endingHashKey().max(rightLeaseHashRange.endingHashKey())); + } else { + // We have non-overlapping leases when rangeDiff is positive. signum() will be 1 in this case. + // If rangeDiff is 1, then it is a continuous hash range. If not, there is a hole. + if (!rangeDiff.equals(BigInteger.ONE)) { + LOG.error("Incomplete hash range found between " + leftmostLeaseToReportInCaseOfHole + + " and " + rightLease); + return Optional.of(new HashRangeHole(leftmostLeaseToReportInCaseOfHole.getHashKeyRange(), + rightLease.getHashKeyRange())); + } + + leftmostLeaseToReportInCaseOfHole = rightLease; + leftLeaseHashRange = rightLeaseHashRange; + } + } + } + + return Optional.empty(); + } + + @VisibleForTesting + static List sortLeasesByHashRange(List leasesWithHashKeyRanges) { + if (leasesWithHashKeyRanges.size() == 0 || leasesWithHashKeyRanges.size() == 1) { + return leasesWithHashKeyRanges; + } + Collections.sort(leasesWithHashKeyRanges, new HashKeyRangeComparator()); + return leasesWithHashKeyRanges; + } + + @Value + @Accessors(fluent = true) + @VisibleForTesting + static class ShardSyncResponse { + private final boolean shouldDoShardSync; + private final String reasonForDecision; + } + + @Value + private static class HashRangeHole { + private final HashKeyRangeForLease hashRangeAtStartOfPossibleHole; + private final HashKeyRangeForLease hashRangeAtEndOfPossibleHole; + + HashRangeHole() { + hashRangeAtStartOfPossibleHole = hashRangeAtEndOfPossibleHole = null; + } + + HashRangeHole(HashKeyRangeForLease hashRangeAtStartOfPossibleHole, + HashKeyRangeForLease hashRangeAtEndOfPossibleHole) { + this.hashRangeAtStartOfPossibleHole = hashRangeAtStartOfPossibleHole; + this.hashRangeAtEndOfPossibleHole = hashRangeAtEndOfPossibleHole; + } + } + + private static class HashRangeHoleTracker { + private HashRangeHole hashRangeHole; + @Getter + private Integer numConsecutiveHoles; + + public boolean hashHighConfidenceOfHoleWith(@NonNull HashRangeHole hashRangeHole) { + if (hashRangeHole.equals(this.hashRangeHole)) { + ++this.numConsecutiveHoles; + } else { + this.hashRangeHole = hashRangeHole; + this.numConsecutiveHoles = 1; + } + + return numConsecutiveHoles >= CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; + } + + public void reset() { + this.hashRangeHole = null; + this.numConsecutiveHoles = 0; + } + } + + private static class HashKeyRangeComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + + @Override + public int compare(KinesisClientLease lease, KinesisClientLease otherLease) { + Validate.notNull(lease); + Validate.notNull(otherLease); + Validate.notNull(lease.getHashKeyRange()); + Validate.notNull(otherLease.getHashKeyRange()); + return ComparisonChain.start() + .compare(lease.getHashKeyRange().startingHashKey(), otherLease.getHashKeyRange().startingHashKey()) + .compare(lease.getHashKeyRange().endingHashKey(), otherLease.getHashKeyRange().endingHashKey()) + .result(); + } } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index a86771e3..9cdb71b5 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -577,8 +577,9 @@ public class Worker implements Runnable { /** * Create shard sync strategy and corresponding {@link LeaderDecider} based on provided configs. PERIODIC - * {@link ShardSyncStrategyType} honors custom leaderDeciders for leader election strategy. All other - * {@link ShardSyncStrategyType}s permit only a default single-leader strategy. + * {@link ShardSyncStrategyType} honors custom leaderDeciders for leader election strategy, and does not permit + * skipping shard syncs if the hash range is complete. All other {@link ShardSyncStrategyType}s permit only a + * default single-leader strategy, and will skip shard syncs unless a hole in the hash range is detected. */ private void createShardSyncStrategy(ShardSyncStrategyType strategyType, LeaderDecider leaderDecider, @@ -587,7 +588,7 @@ public class Worker implements Runnable { case PERIODIC: this.leaderDecider = getOrCreateLeaderDecider(leaderDecider); this.leaderElectedPeriodicShardSyncManager = - getOrCreatePeriodicShardSyncManager(periodicShardSyncManager); + getOrCreatePeriodicShardSyncManager(periodicShardSyncManager, false); this.shardSyncStrategy = createPeriodicShardSyncStrategy(); break; case SHARD_END: @@ -598,7 +599,7 @@ public class Worker implements Runnable { } this.leaderDecider = getOrCreateLeaderDecider(null); this.leaderElectedPeriodicShardSyncManager = - getOrCreatePeriodicShardSyncManager(periodicShardSyncManager); + getOrCreatePeriodicShardSyncManager(periodicShardSyncManager, true); this.shardSyncStrategy = createShardEndShardSyncStrategy(); } @@ -1255,9 +1256,10 @@ public class Worker implements Runnable { Executors.newSingleThreadScheduledExecutor(), PERIODIC_SHARD_SYNC_MAX_WORKERS_DEFAULT); } - private PeriodicShardSyncManager getOrCreatePeriodicShardSyncManager(PeriodicShardSyncManager periodicShardSyncManager) { - // TODO: Configure periodicShardSyncManager with either mandatory shard sync (PERIODIC) or hash range - // validation based shard sync (SHARD_END) based on configured shard sync strategy + /** A non-null PeriodicShardSyncManager can only provided from unit tests. Any application code will create the + * PeriodicShardSyncManager for the first time here. */ + private PeriodicShardSyncManager getOrCreatePeriodicShardSyncManager(PeriodicShardSyncManager periodicShardSyncManager, + boolean isAuditorMode) { if (periodicShardSyncManager != null) { return periodicShardSyncManager; } @@ -1272,7 +1274,10 @@ public class Worker implements Runnable { SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, shardSyncer, null), - metricsFactory); + metricsFactory, + leaseCoordinator.getLeaseManager(), + streamConfig.getStreamProxy(), + isAuditorMode); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java index e817e0ea..92d1e71a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ExtendedSequenceNumber.java @@ -141,6 +141,10 @@ public class ExtendedSequenceNumber implements Comparable parentShardIds = new HashSet(); + private Set childShardIds = new HashSet(); + private HashKeyRangeForLease hashKeyRangeForLease; public KinesisClientLease() { @@ -41,17 +43,22 @@ public class KinesisClientLease extends Lease { this.pendingCheckpoint = other.getPendingCheckpoint(); this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint(); this.parentShardIds.addAll(other.getParentShardIds()); + this.childShardIds = other.getChildShardIds(); + this.hashKeyRangeForLease = other.getHashKeyRange(); } KinesisClientLease(String leaseKey, String leaseOwner, Long leaseCounter, UUID concurrencyToken, Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint, - Long ownerSwitchesSinceCheckpoint, Set parentShardIds) { + Long ownerSwitchesSinceCheckpoint, Set parentShardIds, Set childShardIds, + HashKeyRangeForLease hashKeyRangeForLease) { super(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos); this.checkpoint = checkpoint; this.pendingCheckpoint = pendingCheckpoint; this.ownerSwitchesSinceCheckpoint = ownerSwitchesSinceCheckpoint; this.parentShardIds.addAll(parentShardIds); + this.childShardIds.addAll(childShardIds); + this.hashKeyRangeForLease = hashKeyRangeForLease; } /** @@ -100,6 +107,20 @@ public class KinesisClientLease extends Lease { return new HashSet(parentShardIds); } + /** + * @return shardIds that are the children of this lease. Used for resharding. + */ + public Set getChildShardIds() { + return new HashSet(childShardIds); + } + + /** + * @return hash key range that this lease covers. + */ + public HashKeyRangeForLease getHashKeyRange() { + return hashKeyRangeForLease; + } + /** * Sets checkpoint. * @@ -142,6 +163,29 @@ public class KinesisClientLease extends Lease { this.parentShardIds.clear(); this.parentShardIds.addAll(parentShardIds); } + + /** + * Sets childShardIds. + * + * @param childShardIds may not be null + */ + public void setChildShardIds(Collection childShardIds) { + verifyNotNull(childShardIds, "childShardIds should not be null"); + + this.childShardIds.clear(); + this.childShardIds.addAll(childShardIds); + } + + /** + * Sets hashKeyRangeForLease. + * + * @param hashKeyRangeForLease may not be null + */ + public void setHashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) { + verifyNotNull(hashKeyRangeForLease, "hashKeyRangeForLease should not be null"); + + this.hashKeyRangeForLease = hashKeyRangeForLease; + } private void verifyNotNull(Object object, String message) { if (object == null) { diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 4006e052..6bf9bc58 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -15,6 +15,7 @@ package com.amazonaws.services.kinesis.leases.impl; import java.util.Collection; +import java.util.HashMap; import java.util.Map; import com.amazonaws.services.dynamodbv2.model.AttributeAction; @@ -26,8 +27,11 @@ import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseSerializer; import com.amazonaws.services.kinesis.leases.util.DynamoUtils; +import com.amazonaws.util.CollectionUtils; import com.google.common.base.Strings; +import static com.amazonaws.services.kinesis.leases.impl.UpdateField.HASH_KEY_RANGE; + /** * An implementation of ILeaseSerializer for KinesisClientLease objects. */ @@ -39,6 +43,9 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer getDynamoUpdateLeaseUpdate(KinesisClientLease lease, + UpdateField updateField) { + Map result = new HashMap<>(); + + switch (updateField) { + case CHILD_SHARDS: + // TODO: Implement update fields for child shards + break; + case HASH_KEY_RANGE: + if (lease.getHashKeyRange() != null) { + result.put(STARTING_HASH_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue( + lease.getHashKeyRange().serializedStartingHashKey()), AttributeAction.PUT)); + result.put(ENDING_HASH_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue( + lease.getHashKeyRange().serializedEndingHashKey()), AttributeAction.PUT)); + } + break; + } + + return result; + } + @Override public Collection getKeySchema() { return baseSerializer.getKeySchema(); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index 1e747235..7fe4551a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -590,6 +590,35 @@ public class LeaseManager implements ILeaseManager { return true; } + /** + * {@inheritDoc} + */ + @Override + public void updateLeaseWithMetaInfo(T lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + verifyNotNull(lease, "lease cannot be null"); + verifyNotNull(updateField, "updateField cannot be null"); + + if (LOG.isDebugEnabled()) { + LOG.debug("Updating lease " + lease + " for field " + updateField); + } + + UpdateItemRequest request = new UpdateItemRequest(); + request.setTableName(table); + request.setKey(serializer.getDynamoHashKey(lease)); + request.setExpected(serializer.getDynamoLeaseCounterExpectation(lease)); + + Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); + updates.putAll(serializer.getDynamoUpdateLeaseUpdate(lease)); + request.setAttributeUpdates(updates); + + try { + dynamoDBClient.updateItem(request); + } catch (AmazonClientException e) { + throw convertAndRethrowExceptions("update", lease.getLeaseKey(), e); + } + } + /* * This method contains boilerplate exception handling - it throws or returns something to be thrown. The * inconsistency there exists to satisfy the compiler when this method is used at the end of non-void methods. diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java index 46c45c24..b02ed34c 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java @@ -177,6 +177,12 @@ public class LeaseSerializer implements ILeaseSerializer { return new HashMap(); } + @Override + public Map getDynamoUpdateLeaseUpdate(Lease lease, UpdateField updateField) { + // There is no application-specific data in Lease - just return a map that increments the counter. + return new HashMap(); + } + @Override public Collection getKeySchema() { List keySchema = new ArrayList(); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/UpdateField.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/UpdateField.java new file mode 100644 index 00000000..d621999b --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/UpdateField.java @@ -0,0 +1,26 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.amazonaws.services.kinesis.leases.impl; + +/** + * These are the special fields that will be updated only once during the lifetime of the lease. + * Since these are meta information that will not affect lease ownership or data durability, we allow + * any elected leader or worker to set these fields directly without any conditional checks. + * Note that though HASH_KEY_RANGE will be available during lease initialization in newer versions, we keep this + * for backfilling while rolling forward to newer versions. + */ +public enum UpdateField { + CHILD_SHARDS, HASH_KEY_RANGE +} diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java index 4de54607..1b63bd01 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseManager.java @@ -20,6 +20,7 @@ import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; import com.amazonaws.services.kinesis.leases.impl.Lease; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; /** * Supports basic CRUD operations for Leases. @@ -180,6 +181,19 @@ public interface ILeaseManager { public boolean updateLease(T lease) throws DependencyException, InvalidStateException, ProvisionedThroughputException; + /** + * Update application-specific fields of the given lease in DynamoDB. Does not update fields managed by the leasing + * library such as leaseCounter, leaseOwner, or leaseKey. + ** + * @throws InvalidStateException if lease table does not exist + * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity + * @throws DependencyException if DynamoDB update fails in an unexpected way + */ + default void updateLeaseWithMetaInfo(T lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throw new UnsupportedOperationException("updateLeaseWithMetaInfo is not implemented."); + } + /** * Check (synchronously) if there are any leases in the lease table. * diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java index 35a8fc15..58eb6613 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java @@ -23,6 +23,7 @@ import com.amazonaws.services.dynamodbv2.model.AttributeValueUpdate; import com.amazonaws.services.dynamodbv2.model.ExpectedAttributeValue; import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; import com.amazonaws.services.kinesis.leases.impl.Lease; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; /** * Utility class that manages the mapping of Lease objects/operations to records in DynamoDB. @@ -104,6 +105,15 @@ public interface ILeaseSerializer { */ public Map getDynamoUpdateLeaseUpdate(T lease); + /** + * @param lease + * @param updateField + * @return the attribute value map that updates application-specific data for a lease + */ + default Map getDynamoUpdateLeaseUpdate(T lease, UpdateField updateField) { + throw new UnsupportedOperationException(); + } + /** * @return the key schema for creating a DynamoDB table to store leases */ diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java index c8e820bb..e7b6c285 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java @@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.Arrays; import java.util.List; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -53,6 +54,7 @@ class ExceptionThrowingLeaseManager implements ILeaseManager DELETELEASE(9), DELETEALL(10), UPDATELEASE(11), + UPDATELEASEWITHMETAINFO(12), NONE(Integer.MIN_VALUE); private Integer index; @@ -197,6 +199,14 @@ class ExceptionThrowingLeaseManager implements ILeaseManager return leaseManager.updateLease(lease); } + @Override + public void updateLeaseWithMetaInfo(KinesisClientLease lease, UpdateField updateField) + throws DependencyException, InvalidStateException, ProvisionedThroughputException { + throwExceptions("updateLeaseWithMetaInfo", ExceptionThrowingLeaseManagerMethods.UPDATELEASEWITHMETAINFO); + + leaseManager.updateLeaseWithMetaInfo(lease, updateField); + } + @Override public KinesisClientLease getLease(String shardId) throws DependencyException, InvalidStateException, ProvisionedThroughputException { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java new file mode 100644 index 00000000..fa628b37 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java @@ -0,0 +1,613 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease; +import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; +import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.model.HashKeyRange; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.util.CollectionUtils; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MAX_HASH_KEY; +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MIN_HASH_KEY; +import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.deserialize; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class PeriodicShardSyncManagerTest { + + private static final String WORKER_ID = "workerId"; + + /** Manager for PERIODIC shard sync strategy */ + private PeriodicShardSyncManager periodicShardSyncManager; + + /** Manager for SHARD_END shard sync strategy */ + private PeriodicShardSyncManager auditorPeriodicShardSyncManager; + + @Mock + private LeaderDecider leaderDecider; + @Mock + private ShardSyncTask shardSyncTask; + @Mock + private ILeaseManager leaseManager; + @Mock + private IKinesisProxy kinesisProxy; + + private IMetricsFactory metricsFactory = new NullMetricsFactory(); + + @Before + public void setup() { + periodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask, + metricsFactory, leaseManager, kinesisProxy, false); + auditorPeriodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask, + metricsFactory, leaseManager, kinesisProxy, true); + } + + @Test + public void testForFailureWhenHashRangesAreIncomplete() { + List hashRanges = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("25", MAX_HASH_KEY.toString())); // Missing interval here + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertTrue(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenHashRangesAreComplete() { + List hashRanges = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenUnsortedHashRangesAreComplete() { + List hashRanges = new ArrayList() {{ + add(deserialize("4", "23")); + add(deserialize("2", "3")); + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("24", MAX_HASH_KEY.toString())); + add(deserialize("6", "23")); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(hashRanges).isPresent()); + } + + @Test + public void testForSuccessWhenHashRangesAreCompleteForOverlappingLeasesAtEnd() { + List hashRanges = new ArrayList() {{ + 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())); + add(deserialize("24", "45")); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + Assert.assertFalse(PeriodicShardSyncManager + .checkForHoleInHashKeyRanges(hashRanges).isPresent()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenNoLeasesArePassed() throws Exception { + when(leaseManager.listLeases()).thenReturn(null); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + + } + + @Test + public void testIfShardSyncIsInitiatedWhenEmptyLeasesArePassed() throws Exception { + when(leaseManager.listLeases()).thenReturn(Collections.emptyList()); + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsNotReached() throws Exception { + List leases = 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 -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + } + + @Test + public void testIfShardSyncIsInitiatedWhenConfidenceFactorIsReached() throws Exception { + List leases = 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 -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenHoleIsDueToShardEnd() throws Exception { + List leases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("6", "23")); // Introducing hole here through SHARD_END checkpoint + add(deserialize("4", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + if (lease.getHashKeyRange().startingHashKey().toString().equals("4")) { + lease.setCheckpoint(ExtendedSequenceNumber.SHARD_END); + } else { + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenNoLeasesAreUsedDueToShardEnd() throws Exception { + List leases = new ArrayList() {{ + add(deserialize(MIN_HASH_KEY.toString(), "1")); + add(deserialize("2", "3")); + add(deserialize("4", "23")); + add(deserialize("6", "23")); + add(deserialize("24", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.SHARD_END); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenHoleShifts() throws Exception { + List leases1 = 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 -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases1); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + List leases2 = 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("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + // Resetting the holes + when(leaseManager.listLeases()).thenReturn(leases2); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + @Test + public void testIfShardSyncIsInitiatedWhenHoleShiftsMoreThanOnce() throws Exception { + List leases1 = 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 -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases1); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + List leases2 = 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("25", MAX_HASH_KEY.toString())); + }}.stream().map(hashKeyRangeForLease -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setHashKeyRange(hashKeyRangeForLease); + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + // Resetting the holes + when(leaseManager.listLeases()).thenReturn(leases2); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + // Resetting the holes again + when(leaseManager.listLeases()).thenReturn(leases1); + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + + @Test + public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSync() throws Exception { + 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(hashKeyRange -> new Shard() + .withShardId("shard-" + ++shardCounter[0]) + .withHashKeyRange(new HashKeyRange() + .withStartingHashKey(hashKeyRange.serializedStartingHashKey()) + .withEndingHashKey(hashKeyRange.serializedEndingHashKey()))) + .collect(Collectors.toList()); + + when(kinesisProxy.getShardList()).thenReturn(kinesisShards); + + final int[] leaseCounter = { 0 }; + List leases = hashKeyRangeForLeases.stream() + .map(hashKeyRange -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey("shard-" + ++leaseCounter[0]); + // Setting the hash range only for the last two leases + if (leaseCounter[0] >= 3) { + lease.setHashKeyRange(hashKeyRange); + } + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases); + + // Assert that SHARD_END shard sync should never trigger, but PERIODIC shard sync should always trigger + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + + // Assert that all the leases now have hash ranges set + for (KinesisClientLease lease : leases) { + Assert.assertNotNull(lease.getHashKeyRange()); + } + } + + @Test + public void testIfMissingHashRangeInformationIsFilledBeforeEvaluatingForShardSyncInHoleScenario() throws Exception { + 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(hashKeyRange -> new Shard() + .withShardId("shard-" + ++shardCounter[0]) + .withHashKeyRange(new HashKeyRange() + .withStartingHashKey(hashKeyRange.serializedStartingHashKey()) + .withEndingHashKey(hashKeyRange.serializedEndingHashKey()))) + .collect(Collectors.toList()); + + when(kinesisProxy.getShardList()).thenReturn(kinesisShards); + + final int[] leaseCounter = { 0 }; + List leases = hashKeyRangeForLeases.stream() + .map(hashKeyRange -> { + KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey("shard-" + ++leaseCounter[0]); + // Setting the hash range only for the last two leases + if (leaseCounter[0] >= 3) { + lease.setHashKeyRange(hashKeyRange); + } + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return lease; + }).collect(Collectors.toList()); + + when(leaseManager.listLeases()).thenReturn(leases); + + // Assert that shard sync should trigger after breaching threshold + for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + } + Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + Assert.assertTrue(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); + + // Assert that all the leases now have hash ranges set + for (KinesisClientLease lease : leases) { + Assert.assertNotNull(lease.getHashKeyRange()); + } + } + + @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); + Assert.assertFalse(periodicShardSyncManager.hasHoleInLeases(leases).isPresent()); + Assert.assertFalse(auditorPeriodicShardSyncManager.hasHoleInLeases(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(leases).isPresent()); + Assert.assertFalse(auditorPeriodicShardSyncManager.hasHoleInLeases(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(leases).isPresent()); + Assert.assertFalse(auditorPeriodicShardSyncManager.hasHoleInLeases(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 KinesisClientLease lease = new KinesisClientLease(); + long leaseEndKey; + if (i != initialShardCount) { + leaseEndKey = (hashRangeInternalMax / initialShardCount) * i; + lease.setHashKeyRange(deserialize(leaseStartKey + "", leaseEndKey + "")); + } else { + leaseEndKey = 0; + lease.setHashKeyRange(deserialize(leaseStartKey + "", MAX_HASH_KEY.toString())); + } + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + lease.setLeaseKey("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.getChildShardIds())).collect(Collectors.toList()); + + int leasesToMerge = (int) ((leasesEligibleForMerge.size() - 1) / 2.0 * Math.random()); + for (int i = 0; i < leasesToMerge; i += 2) { + KinesisClientLease parent1 = leasesEligibleForMerge.get(i); + KinesisClientLease parent2 = leasesEligibleForMerge.get(i + 1); + if (parent2.getHashKeyRange().startingHashKey() + .subtract(parent1.getHashKeyRange().endingHashKey()).equals(BigInteger.ONE)) { + parent1.setCheckpoint(ExtendedSequenceNumber.SHARD_END); + + if (!shouldKeepSomeParentsInProgress || (shouldKeepSomeParentsInProgress && isOneFromDiceRoll())) { + parent2.setCheckpoint(ExtendedSequenceNumber.SHARD_END); + } + + KinesisClientLease child = new KinesisClientLease(); + child.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child.setLeaseKey("shard-" + ++leaseCounter); + child.setHashKeyRange(new HashKeyRangeForLease(parent1.getHashKeyRange().startingHashKey(), + parent2.getHashKeyRange().endingHashKey())); + parent1.setChildShardIds(Collections.singletonList(child.getLeaseKey())); + parent2.setChildShardIds(Collections.singletonList(child.getLeaseKey())); + child.setParentShardIds(Sets.newHashSet(parent1.getLeaseKey(), parent2.getLeaseKey())); + + initialLeases.add(child); + } + } + + return leaseCounter; + } + + private int split(List initialLeases, int leaseCounter) { + List leasesEligibleForSplit = initialLeases.stream() + .filter(l -> CollectionUtils.isNullOrEmpty(l.getChildShardIds())).collect(Collectors.toList()); + + int leasesToSplit = (int) (leasesEligibleForSplit.size() * Math.random()); + for (int i = 0; i < leasesToSplit; i++) { + KinesisClientLease parent = leasesEligibleForSplit.get(i); + parent.setCheckpoint(ExtendedSequenceNumber.SHARD_END); + + KinesisClientLease child1 = new KinesisClientLease(); + child1.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child1.setHashKeyRange(new HashKeyRangeForLease(parent.getHashKeyRange().startingHashKey(), + parent.getHashKeyRange().startingHashKey().add(parent.getHashKeyRange().endingHashKey()) + .divide(new BigInteger("2")))); + child1.setLeaseKey("shard-" + ++leaseCounter); + + KinesisClientLease child2 = new KinesisClientLease(); + child2.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + child2.setHashKeyRange(new HashKeyRangeForLease(parent.getHashKeyRange().startingHashKey() + .add(parent.getHashKeyRange().endingHashKey()).divide(new BigInteger("2")).add(BigInteger.ONE), + parent.getHashKeyRange().endingHashKey())); + child2.setLeaseKey("shard-" + ++leaseCounter); + + child1.setParentShardIds(Sets.newHashSet(parent.getLeaseKey())); + child2.setParentShardIds(Sets.newHashSet(parent.getLeaseKey())); + parent.setChildShardIds(Lists.newArrayList(child1.getLeaseKey(), child2.getLeaseKey())); + + initialLeases.add(child1); + initialLeases.add(child2); + } + + return leaseCounter; + } + + private boolean isHeads() { + return Math.random() <= 0.5; + } + + private boolean isOneFromDiceRoll() { + return Math.random() <= 0.16; + } + + private enum ReshardType { + SPLIT, + MERGE, + ANY + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index f6bfdd73..b7e1dd51 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -174,6 +174,8 @@ public class WorkerTest { @Mock private IKinesisProxy proxy; @Mock + private StreamConfig streamConfig; + @Mock private WorkerThreadPoolExecutor executorService; @Mock private WorkerCWMetricsFactory cwMetricsFactory; @@ -200,6 +202,8 @@ public class WorkerTest { config.withMaxInitializationAttempts(1); recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); + when(leaseCoordinator.getLeaseManager()).thenReturn(mock(ILeaseManager.class)); + when(streamConfig.getStreamProxy()).thenReturn(kinesisProxy); } // CHECKSTYLE:IGNORE AnonInnerLengthCheck FOR NEXT 50 LINES @@ -257,7 +261,6 @@ public class WorkerTest { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); - IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; int idleTimeInMilliseconds = 1000; @@ -306,7 +309,6 @@ public class WorkerTest { public void testWorkerLoopWithCheckpoint() { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; - IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; int idleTimeInMilliseconds = 1000; @@ -376,7 +378,6 @@ public class WorkerTest { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; config = new KinesisClientLibConfiguration(stageName, null, null, WORKER_ID); - IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; int idleTimeInMilliseconds = 1000; @@ -866,7 +867,6 @@ public class WorkerTest { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); @@ -954,7 +954,6 @@ public class WorkerTest { public void testShutdownCallableNotAllowedTwice() throws Exception { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); @@ -1019,7 +1018,6 @@ public class WorkerTest { public void testGracefulShutdownSingleFuture() throws Exception { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); @@ -1107,7 +1105,6 @@ public class WorkerTest { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); @@ -1190,7 +1187,6 @@ public class WorkerTest { when(completedLease.getParentShardIds()).thenReturn(Collections.singleton(parentShardId)); when(completedLease.getCheckpoint()).thenReturn(ExtendedSequenceNumber.SHARD_END); when(completedLease.getConcurrencyToken()).thenReturn(UUID.randomUUID()); - final StreamConfig streamConfig = mock(StreamConfig.class); final IMetricsFactory metricsFactory = mock(IMetricsFactory.class); final List leases = Collections.singletonList(completedLease); final List currentAssignments = new ArrayList<>(); @@ -1238,7 +1234,6 @@ public class WorkerTest { public void testRequestShutdownWithLostLease() throws Exception { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); @@ -1351,7 +1346,6 @@ public class WorkerTest { public void testRequestShutdownWithAllLeasesLost() throws Exception { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); @@ -1469,7 +1463,6 @@ public class WorkerTest { public void testLeaseCancelledAfterShutdownRequest() throws Exception { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); @@ -1553,7 +1546,6 @@ public class WorkerTest { public void testEndOfShardAfterShutdownRequest() throws Exception { IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); - StreamConfig streamConfig = mock(StreamConfig.class); IMetricsFactory metricsFactory = mock(IMetricsFactory.class); ExtendedSequenceNumber checkpoint = new ExtendedSequenceNumber("123", 0L); diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java index c029926c..4f4fdbca 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java @@ -30,6 +30,8 @@ public class KinesisClientLeaseBuilder { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); + private HashKeyRangeForLease hashKeyRangeForLease; public KinesisClientLeaseBuilder withLeaseKey(String leaseKey) { this.leaseKey = leaseKey; @@ -76,8 +78,19 @@ public class KinesisClientLeaseBuilder { return this; } + public KinesisClientLeaseBuilder withChildShardIds(Set childShardIds) { + this.childShardIds = childShardIds; + return this; + } + + public KinesisClientLeaseBuilder withHashKeyRange(HashKeyRangeForLease hashKeyRangeForLease) { + this.hashKeyRangeForLease = hashKeyRangeForLease; + return this; + } + public KinesisClientLease build() { return new KinesisClientLease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds); + checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, + hashKeyRangeForLease); } } \ No newline at end of file From 57f8c120d04893465877ea31dcefc06f4f68c822 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 2 Jul 2020 10:15:44 -0700 Subject: [PATCH 07/43] Periodc Shard Sync Metrics and Config --- .../worker/KinesisClientLibConfiguration.java | 48 +++++++++++++++++ .../lib/worker/PeriodicShardSyncManager.java | 51 +++++++++++++------ .../clientlibrary/lib/worker/Worker.java | 28 +++++----- .../worker/PeriodicShardSyncManagerTest.java | 31 ++++++----- 4 files changed, 115 insertions(+), 43 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 86e7a496..d5749133 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -175,6 +175,16 @@ public class KinesisClientLibConfiguration { */ public static final ShardSyncStrategyType DEFAULT_SHARD_SYNC_STRATEGY_TYPE = ShardSyncStrategyType.SHARD_END; + /** + * Default Lease Recovery Auditor execution period for SHARD_END ShardSyncStrategyType. + */ + public static final long LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS = 2 * 60 * 1000L; + + /** + * Default Lease Recovery Auditor inconsistency confidence threshold for running full shard sync for HARD_END ShardSyncStrategyType. + */ + public static final int LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD = 3; + /** * Default Shard prioritization strategy. */ @@ -246,6 +256,8 @@ public class KinesisClientLibConfiguration { private ShardPrioritization shardPrioritization; private long shutdownGraceMillis; private ShardSyncStrategyType shardSyncStrategyType; + private long leasesRecoveryAuditorExecutionFrequencyMillis; + private int leasesRecoveryAuditorInconsistencyConfidenceThreshold; @Getter private Optional timeoutInSeconds = Optional.empty(); @@ -515,6 +527,8 @@ public class KinesisClientLibConfiguration { InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; this.shardSyncStrategyType = DEFAULT_SHARD_SYNC_STRATEGY_TYPE; + this.leasesRecoveryAuditorExecutionFrequencyMillis = LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS; + this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; this.recordsFetcherFactory = new SimpleRecordsFetcherFactory(); this.billingMode = billingMode; @@ -625,6 +639,8 @@ public class KinesisClientLibConfiguration { InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; this.shardSyncStrategyType = DEFAULT_SHARD_SYNC_STRATEGY_TYPE; + this.leasesRecoveryAuditorExecutionFrequencyMillis = LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS; + this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; this.recordsFetcherFactory = recordsFetcherFactory; this.shutdownGraceMillis = shutdownGraceMillis; @@ -872,6 +888,20 @@ public class KinesisClientLibConfiguration { return shardSyncStrategyType; } + /** + * @return leasesRecoveryAuditorExecutionFrequencyMillis to be used by SHARD_END ShardSyncStrategyType. + */ + public long getLeasesRecoveryAuditorExecutionFrequencyMillis() { + return leasesRecoveryAuditorExecutionFrequencyMillis; + } + + /** + * @return leasesRecoveryAuditorInconsistencyConfidenceThreshold to be used by SHARD_END ShardSyncStrategyType. + */ + public int getLeasesRecoveryAuditorInconsistencyConfidenceThreshold() { + return leasesRecoveryAuditorInconsistencyConfidenceThreshold; + } + /** * @return Max leases this Worker can handle at a time */ @@ -1249,6 +1279,24 @@ public class KinesisClientLibConfiguration { return this; } + /** + * @param leasesRecoveryAuditorExecutionFrequencyMillis Leases Recovery Auditor Execution period. + * @return {@link KinesisClientLibConfiguration} + */ + public KinesisClientLibConfiguration withLeasesRecoveryAuditorExecutionFrequencyMillis(long leasesRecoveryAuditorExecutionFrequencyMillis) { + this.leasesRecoveryAuditorExecutionFrequencyMillis = leasesRecoveryAuditorExecutionFrequencyMillis; + return this; + } + + /** + * @param leasesRecoveryAuditorInconsistencyConfidenceThreshold Leases Recovery Auditor Execution inconsistency confidence threshold. + * @return {@link KinesisClientLibConfiguration} + */ + public KinesisClientLibConfiguration withLeasesRecoveryAuditorInconsistencyConfidenceThreshold(int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { + this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold; + return this; + } + /** * * @param regionName The region name for the service diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java index 75a747e6..32fdec54 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java @@ -34,7 +34,9 @@ import com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; import com.amazonaws.services.kinesis.leases.impl.UpdateField; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; +import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.util.CollectionUtils; import com.google.common.annotations.VisibleForTesting; @@ -65,16 +67,12 @@ class PeriodicShardSyncManager { /** DEFAULT interval is used for PERIODIC {@link ShardSyncStrategyType}. */ private static final long DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 1000L; - /** AUDITOR interval is used for non-PERIODIC {@link ShardSyncStrategyType} auditor processes. */ - private static final long AUDITOR_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS = 2 * 60 * 1000L; - /** Parameters for validating hash range completeness when running in auditor mode. */ @VisibleForTesting static final BigInteger MIN_HASH_KEY = BigInteger.ZERO; @VisibleForTesting static final BigInteger MAX_HASH_KEY = new BigInteger("2").pow(128).subtract(BigInteger.ONE); - @VisibleForTesting - static final int CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY = 3; + static final String PERIODIC_SHARD_SYNC_MANAGER = "PeriodicShardSyncManager"; private final HashRangeHoleTracker hashRangeHoleTracker = new HashRangeHoleTracker(); private final String workerId; @@ -86,6 +84,9 @@ class PeriodicShardSyncManager { private final boolean isAuditorMode; private final long periodicShardSyncIntervalMillis; private boolean isRunning; + private final IMetricsFactory metricsFactory; + private final int leasesRecoveryAuditorInconsistencyConfidenceThreshold; + PeriodicShardSyncManager(String workerId, LeaderDecider leaderDecider, @@ -93,9 +94,12 @@ class PeriodicShardSyncManager { IMetricsFactory metricsFactory, ILeaseManager leaseManager, IKinesisProxy kinesisProxy, - boolean isAuditorMode) { + boolean isAuditorMode, + long leasesRecoveryAuditorExecutionFrequencyMillis, + int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { this(workerId, leaderDecider, shardSyncTask, Executors.newSingleThreadScheduledExecutor(), metricsFactory, - leaseManager, kinesisProxy, isAuditorMode); + leaseManager, kinesisProxy, isAuditorMode, leasesRecoveryAuditorExecutionFrequencyMillis, + leasesRecoveryAuditorInconsistencyConfidenceThreshold); } PeriodicShardSyncManager(String workerId, @@ -105,7 +109,9 @@ class PeriodicShardSyncManager { IMetricsFactory metricsFactory, ILeaseManager leaseManager, IKinesisProxy kinesisProxy, - boolean isAuditorMode) { + boolean isAuditorMode, + long leasesRecoveryAuditorExecutionFrequencyMillis, + int leasesRecoveryAuditorInconsistencyConfidenceThreshold) { Validate.notBlank(workerId, "WorkerID is required to initialize PeriodicShardSyncManager."); Validate.notNull(leaderDecider, "LeaderDecider is required to initialize PeriodicShardSyncManager."); Validate.notNull(shardSyncTask, "ShardSyncTask is required to initialize PeriodicShardSyncManager."); @@ -115,11 +121,13 @@ class PeriodicShardSyncManager { this.shardSyncThreadPool = shardSyncThreadPool; this.leaseManager = leaseManager; this.kinesisProxy = kinesisProxy; + this.metricsFactory = metricsFactory; this.isAuditorMode = isAuditorMode; + this.leasesRecoveryAuditorInconsistencyConfidenceThreshold = leasesRecoveryAuditorInconsistencyConfidenceThreshold; if (isAuditorMode) { Validate.notNull(this.leaseManager, "LeaseManager is required for non-PERIODIC shard sync strategies."); Validate.notNull(this.kinesisProxy, "KinesisProxy is required for non-PERIODIC shard sync strategies."); - this.periodicShardSyncIntervalMillis = AUDITOR_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS; + this.periodicShardSyncIntervalMillis = leasesRecoveryAuditorExecutionFrequencyMillis; } else { this.periodicShardSyncIntervalMillis = DEFAULT_PERIODIC_SHARD_SYNC_INTERVAL_MILLIS; } @@ -166,8 +174,14 @@ class PeriodicShardSyncManager { if (leaderDecider.isLeader(workerId)) { LOG.debug("WorkerId " + workerId + " is a leader, running the shard sync task"); + MetricsHelper.startScope(metricsFactory, PERIODIC_SHARD_SYNC_MANAGER); + boolean isRunSuccess = false; + final long runStartMillis = System.currentTimeMillis(); + try { final ShardSyncResponse shardSyncResponse = checkForShardSync(); + MetricsHelper.addSuccessAndLatency(runStartMillis, shardSyncResponse.shouldDoShardSync(), MetricsLevel.SUMMARY); + MetricsHelper.addSuccessAndLatency(runStartMillis, shardSyncResponse.isHoleDetected(), MetricsLevel.SUMMARY); if (shardSyncResponse.shouldDoShardSync()) { LOG.info("Periodic shard syncer initiating shard sync due to the reason - " + shardSyncResponse.reasonForDecision()); @@ -175,8 +189,12 @@ class PeriodicShardSyncManager { } else { LOG.info("Skipping shard sync due to the reason - " + shardSyncResponse.reasonForDecision()); } + isRunSuccess = true; } catch (Exception e) { LOG.error("Caught exception while running periodic shard syncer.", e); + } finally { + MetricsHelper.addSuccessAndLatency(runStartMillis, isRunSuccess, MetricsLevel.SUMMARY); + MetricsHelper.endScope(); } } else { LOG.debug("WorkerId " + workerId + " is not a leader, not running the shard sync task"); @@ -189,7 +207,7 @@ class PeriodicShardSyncManager { if (!isAuditorMode) { // If we are running with PERIODIC shard sync strategy, we should sync every time. - return new ShardSyncResponse(true, "Syncing every time with PERIODIC shard sync strategy."); + return new ShardSyncResponse(true, false, "Syncing every time with PERIODIC shard sync strategy."); } // Get current leases from DynamoDB. @@ -198,7 +216,7 @@ class PeriodicShardSyncManager { if (CollectionUtils.isNullOrEmpty(currentLeases)) { // If the current leases are null or empty, then we need to initiate a shard sync. LOG.info("No leases found. Will trigger a shard sync."); - return new ShardSyncResponse(true, "No leases found."); + return new ShardSyncResponse(true, false, "No leases found."); } // Check if there are any holes in the hash range covered by current leases. Return the first hole if present. @@ -210,13 +228,13 @@ class PeriodicShardSyncManager { final boolean hasHoleWithHighConfidence = hashRangeHoleTracker.hashHighConfidenceOfHoleWith(hashRangeHoleOpt.get()); - return new ShardSyncResponse(hasHoleWithHighConfidence, + return new ShardSyncResponse(hasHoleWithHighConfidence, true, "Detected the same hole for " + hashRangeHoleTracker.getNumConsecutiveHoles() + " times. " + - "Will initiate shard sync after reaching threshold: " + CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY); + "Will initiate shard sync after reaching threshold: " + leasesRecoveryAuditorInconsistencyConfidenceThreshold); } else { // If hole is not present, clear any previous hole tracking and return false. hashRangeHoleTracker.reset(); - return new ShardSyncResponse(false, "Hash range is complete."); + return new ShardSyncResponse(false, false, "Hash range is complete."); } } @@ -331,6 +349,7 @@ class PeriodicShardSyncManager { @VisibleForTesting static class ShardSyncResponse { private final boolean shouldDoShardSync; + private final boolean isHoleDetected; private final String reasonForDecision; } @@ -350,7 +369,7 @@ class PeriodicShardSyncManager { } } - private static class HashRangeHoleTracker { + private class HashRangeHoleTracker { private HashRangeHole hashRangeHole; @Getter private Integer numConsecutiveHoles; @@ -363,7 +382,7 @@ class PeriodicShardSyncManager { this.numConsecutiveHoles = 1; } - return numConsecutiveHoles >= CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; + return numConsecutiveHoles >= leasesRecoveryAuditorInconsistencyConfidenceThreshold; } public void reset() { diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 9cdb71b5..8373fa8b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -1265,19 +1265,21 @@ public class Worker implements Runnable { } return new PeriodicShardSyncManager(config.getWorkerIdentifier(), - leaderDecider, - new ShardSyncTask(streamConfig.getStreamProxy(), - leaseCoordinator.getLeaseManager(), - config.getInitialPositionInStreamExtended(), - config.shouldCleanupLeasesUponShardCompletion(), - config.shouldIgnoreUnexpectedChildShards(), - SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, - shardSyncer, - null), - metricsFactory, - leaseCoordinator.getLeaseManager(), - streamConfig.getStreamProxy(), - isAuditorMode); + leaderDecider, + new ShardSyncTask(streamConfig.getStreamProxy(), + leaseCoordinator.getLeaseManager(), + config.getInitialPositionInStreamExtended(), + config.shouldCleanupLeasesUponShardCompletion(), + config.shouldIgnoreUnexpectedChildShards(), + SHARD_SYNC_SLEEP_FOR_PERIODIC_SHARD_SYNC, + shardSyncer, + null), + metricsFactory, + leaseCoordinator.getLeaseManager(), + streamConfig.getStreamProxy(), + isAuditorMode, + config.getLeasesRecoveryAuditorExecutionFrequencyMillis(), + config.getLeasesRecoveryAuditorInconsistencyConfidenceThreshold()); } /** diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java index fa628b37..779ba92f 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManagerTest.java @@ -39,7 +39,6 @@ import java.util.Collections; import java.util.List; import java.util.stream.Collectors; -import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MAX_HASH_KEY; import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.PeriodicShardSyncManager.MIN_HASH_KEY; import static com.amazonaws.services.kinesis.leases.impl.HashKeyRangeForLease.deserialize; @@ -49,6 +48,8 @@ import static org.mockito.Mockito.when; public class PeriodicShardSyncManagerTest { private static final String WORKER_ID = "workerId"; + public static final long LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS = 2 * 60 * 1000L; + public static final int LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD = 3; /** Manager for PERIODIC shard sync strategy */ private PeriodicShardSyncManager periodicShardSyncManager; @@ -70,9 +71,11 @@ public class PeriodicShardSyncManagerTest { @Before public void setup() { periodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask, - metricsFactory, leaseManager, kinesisProxy, false); + metricsFactory, leaseManager, kinesisProxy, false, LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS, + LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD); auditorPeriodicShardSyncManager = new PeriodicShardSyncManager(WORKER_ID, leaderDecider, shardSyncTask, - metricsFactory, leaseManager, kinesisProxy, true); + metricsFactory, leaseManager, kinesisProxy, true, LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS, + LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD); } @Test @@ -179,7 +182,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); when(leaseManager.listLeases()).thenReturn(leases); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -201,7 +204,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); when(leaseManager.listLeases()).thenReturn(leases); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -229,7 +232,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); when(leaseManager.listLeases()).thenReturn(leases); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -253,7 +256,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); when(leaseManager.listLeases()).thenReturn(leases); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -277,7 +280,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); when(leaseManager.listLeases()).thenReturn(leases1); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -297,7 +300,7 @@ public class PeriodicShardSyncManagerTest { // Resetting the holes when(leaseManager.listLeases()).thenReturn(leases2); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -321,7 +324,7 @@ public class PeriodicShardSyncManagerTest { }).collect(Collectors.toList()); when(leaseManager.listLeases()).thenReturn(leases1); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -341,13 +344,13 @@ public class PeriodicShardSyncManagerTest { // Resetting the holes when(leaseManager.listLeases()).thenReturn(leases2); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } // Resetting the holes again when(leaseManager.listLeases()).thenReturn(leases1); - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -392,7 +395,7 @@ public class PeriodicShardSyncManagerTest { when(leaseManager.listLeases()).thenReturn(leases); // Assert that SHARD_END shard sync should never trigger, but PERIODIC shard sync should always trigger - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } @@ -442,7 +445,7 @@ public class PeriodicShardSyncManagerTest { when(leaseManager.listLeases()).thenReturn(leases); // Assert that shard sync should trigger after breaching threshold - for (int i = 1; i < CONSECUTIVE_HOLES_FOR_TRIGGERING_RECOVERY; i++) { + for (int i = 1; i < LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD; i++) { Assert.assertTrue(periodicShardSyncManager.checkForShardSync().shouldDoShardSync()); Assert.assertFalse(auditorPeriodicShardSyncManager.checkForShardSync().shouldDoShardSync()); } From 07606883758f50cab5e816a1b88df1c5a81b560e Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 6 Jul 2020 14:50:21 -0400 Subject: [PATCH 08/43] KinesisShardSyncer only create leases for one level of leases (#49) Co-authored-by: Joshua Kim --- .../lib/worker/KinesisShardSyncer.java | 110 +- .../NonEmptyLeaseTableSynchronizer.java | 73 +- .../worker/ExceptionThrowingLeaseManager.java | 2 +- .../lib/worker/ShardSyncerTest.java | 1514 ++++++++++++++--- 4 files changed, 1376 insertions(+), 323 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index 21890663..358c8136 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -31,6 +31,7 @@ import com.amazonaws.services.kinesis.leases.impl.LeaseManager; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; import com.amazonaws.util.CollectionUtils; +import lombok.NoArgsConstructor; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.commons.lang3.StringUtils; @@ -457,7 +458,7 @@ class KinesisShardSyncer implements ShardSyncer { /** * Note: Package level access for testing purposes only. * Check if this shard is a descendant of a shard that is (or will be) processed. - * Create leases for the ancestors of this shard as required. + * Create leases for the first ancestor of this shard that needs to be processed, as required. * See javadoc of determineNewLeasesToCreate() for rules and example. * * @param shardId The shardId to check. @@ -473,9 +474,10 @@ class KinesisShardSyncer implements ShardSyncer { static boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId, InitialPositionInStreamExtended initialPosition, Set shardIdsOfCurrentLeases, Map shardIdToShardMapOfAllKinesisShards, - Map shardIdToLeaseMapOfNewShards, Map memoizationContext) { + Map shardIdToLeaseMapOfNewShards, MemoizationContext memoizationContext) { + + final Boolean previousValue = memoizationContext.isDescendant(shardId); - Boolean previousValue = memoizationContext.get(shardId); if (previousValue != null) { return previousValue; } @@ -495,10 +497,13 @@ class KinesisShardSyncer implements ShardSyncer { shard = shardIdToShardMapOfAllKinesisShards.get(shardId); parentShardIds = getParentShardIds(shard, shardIdToShardMapOfAllKinesisShards); for (String parentShardId : parentShardIds) { - // Check if the parent is a descendant, and include its ancestors. - if (checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, initialPosition, - shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToLeaseMapOfNewShards, - memoizationContext)) { + // Check if the parent is a descendant, and include its ancestors. Or, if the parent is NOT a + // descendant but we should create a lease for it anyway (e.g. to include in processing from + // TRIM_HORIZON or AT_TIMESTAMP). If either is true, then we mark the current shard as a descendant. + final boolean isParentDescendant = checkIfDescendantAndAddNewLeasesForAncestors(parentShardId, + initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, + shardIdToLeaseMapOfNewShards, memoizationContext); + if (isParentDescendant || memoizationContext.shouldCreateLease(parentShardId)) { isDescendant = true; descendantParentShardIds.add(parentShardId); LOG.debug("Parent shard " + parentShardId + " is a descendant."); @@ -511,37 +516,76 @@ class KinesisShardSyncer implements ShardSyncer { if (isDescendant) { for (String parentShardId : parentShardIds) { if (!shardIdsOfCurrentLeases.contains(parentShardId)) { - LOG.debug("Need to create a lease for shardId " + parentShardId); KinesisClientLease lease = shardIdToLeaseMapOfNewShards.get(parentShardId); + + // If the lease for the parent shard does not already exist, there are two cases in which we + // would want to create it: + // - If we have already marked the parentShardId for lease creation in a prior recursive + // call. This could happen if we are trying to process from TRIM_HORIZON or AT_TIMESTAMP. + // - If the parent shard is not a descendant but the current shard is a descendant, then + // the parent shard is the oldest shard in the shard hierarchy that does not have an + // ancestor in the lease table (the adjacent parent is necessarily a descendant, and + // therefore covered in the lease table). So we should create a lease for the parent. + if (lease == null) { - lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); - shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + if (memoizationContext.shouldCreateLease(parentShardId) || + !descendantParentShardIds.contains(parentShardId)) { + LOG.debug("Need to create a lease for shardId " + parentShardId); + lease = newKCLLease(shardIdToShardMapOfAllKinesisShards.get(parentShardId)); + shardIdToLeaseMapOfNewShards.put(parentShardId, lease); + } } - if (descendantParentShardIds.contains(parentShardId) && !initialPosition - .getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { - lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { - lease.setCheckpoint(convertToCheckpoint(initialPosition)); + /** + * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the + * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will + * add a lease just like we do for TRIM_HORIZON. However we will only return back records + * with server-side timestamp at or after the specified initial position timestamp. + * + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: (4, 5, 7) + * + * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with + * timestamp value 206. We will then create new leases for all the shards 0 and 1 (with + * checkpoint set AT_TIMESTAMP), even though these ancestor shards have an epoch less than + * 206. However as we begin processing the ancestor shards, their checkpoints would be + * updated to SHARD_END and their leases would then be deleted since they won't have records + * with server-side timestamp at/after 206. And after that we will begin processing the + * descendant shards with epoch at/after 206 and we will return the records that meet the + * timestamp requirement for these shards. + */ + if (lease != null) { + if (descendantParentShardIds.contains(parentShardId) && !initialPosition + .getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { + lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + } else { + lease.setCheckpoint(convertToCheckpoint(initialPosition)); + } } } } } else { - // This shard should be included, if the customer wants to process all records in the stream or - // if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a lease just like we do - // for TRIM_HORIZON. However we will only return back records with server-side timestamp at or - // after the specified initial position timestamp. + // This shard is not a descendant, but should still be included if the customer wants to process all + // records in the stream or if the initial position is AT_TIMESTAMP. For AT_TIMESTAMP, we will add a + // lease just like we do for TRIM_HORIZON. However we will only return back records with server-side + // timestamp at or after the specified initial position timestamp. if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON) || initialPosition.getInitialPositionInStream() .equals(InitialPositionInStream.AT_TIMESTAMP)) { - isDescendant = true; + memoizationContext.setShouldCreateLease(shardId, true); } } } } - memoizationContext.put(shardId, isDescendant); + memoizationContext.setIsDescendant(shardId, isDescendant); return isDescendant; } // CHECKSTYLE:ON CyclomaticComplexity @@ -834,4 +878,28 @@ class KinesisShardSyncer implements ShardSyncer { } + /** + * Helper class to pass around state between recursive traversals of shard hierarchy. + */ + @NoArgsConstructor + static class MemoizationContext { + private Map isDescendantMap = new HashMap<>(); + private Map shouldCreateLeaseMap = new HashMap<>(); + + Boolean isDescendant(String shardId) { + return isDescendantMap.get(shardId); + } + + void setIsDescendant(String shardId, Boolean isDescendant) { + isDescendantMap.put(shardId, isDescendant); + } + + Boolean shouldCreateLease(String shardId) { + return shouldCreateLeaseMap.computeIfAbsent(shardId, x -> Boolean.FALSE); + } + + void setShouldCreateLease(String shardId, Boolean shouldCreateLease) { + shouldCreateLeaseMap.put(shardId, shouldCreateLease); + } + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java index 53c42980..2a868d30 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/NonEmptyLeaseTableSynchronizer.java @@ -47,8 +47,10 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { * Note: Package level access only for testing purposes. * * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, - * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): - * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. + * determine if it is a descendant of any shard which is or will be processed (e.g. for which a lease exists): + * If so, create a lease for the first ancestor that needs to be processed (if needed). We will create leases + * for no more than one level in the ancestry tree. Once we find the first ancestor that needs to be processed, + * we will avoid creating leases for further descendants of that ancestor. * If not, set checkpoint of the shard to the initial position specified by the client. * To check if we need to create leases for ancestors, we use the following rules: * * If we began (or will begin) processing data for a shard, then we must reach end of that shard before @@ -67,10 +69,17 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { * 0 1 2 3 4 5 - shards till epoch 102 * \ / \ / | | * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | / \ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) - * New leases to create: (2, 6, 7, 8, 9, 10) + * \ / | / \ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * + * Current leases: (4, 5, 7) + * + * If initial position is LATEST: + * - New leases to create: (6) + * If initial position is TRIM_HORIZON: + * - New leases to create: (0, 1) + * If initial position is AT_TIMESTAMP(epoch=200): + * - New leases to create: (0, 1) * * The leases returned are sorted by the starting sequence number - following the same order * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail @@ -104,7 +113,8 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { } List openShards = KinesisShardSyncer.getOpenShards(shards); - Map memoizationContext = new HashMap<>(); + final KinesisShardSyncer.MemoizationContext memoizationContext = new KinesisShardSyncer.MemoizationContext(); + // Iterate over the open shards and find those that don't have any lease entries. for (Shard shard : openShards) { @@ -115,43 +125,30 @@ class NonEmptyLeaseTableSynchronizer implements LeaseSynchronizer { } else if (inconsistentShardIds.contains(shardId)) { LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease"); } else { - LOG.debug("Need to create a lease for shardId " + shardId); - KinesisClientLease newLease = KinesisShardSyncer.newKCLLease(shard); + LOG.debug("Beginning traversal of ancestry tree for shardId " + shardId); + + // A shard is a descendant if at least one if its ancestors exists in the lease table. + // We will create leases for only one level in the ancestry tree. Once we find the first ancestor + // that needs to be processed in order to complete the hash range, we will not create leases for + // further descendants of that ancestor. boolean isDescendant = KinesisShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, initialPosition, shardIdsOfCurrentLeases, shardIdToShardMapOfAllKinesisShards, shardIdToNewLeaseMap, memoizationContext); - /** - * If the shard is a descendant and the specified initial position is AT_TIMESTAMP, then the - * checkpoint should be set to AT_TIMESTAMP, else to TRIM_HORIZON. For AT_TIMESTAMP, we will add a - * lease just like we do for TRIM_HORIZON. However we will only return back records with server-side - * timestamp at or after the specified initial position timestamp. - * - * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5 - shards till epoch 102 - * \ / \ / | | - * 6 7 4 5 - shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * - * Current leases: empty set - * - * For the above example, suppose the initial position in stream is set to AT_TIMESTAMP with - * timestamp value 206. We will then create new leases for all the shards (with checkpoint set to - * AT_TIMESTAMP), including the ancestor shards with epoch less than 206. However as we begin - * processing the ancestor shards, their checkpoints would be updated to SHARD_END and their leases - * would then be deleted since they won't have records with server-side timestamp at/after 206. And - * after that we will begin processing the descendant shards with epoch at/after 206 and we will - * return the records that meet the timestamp requirement for these shards. - */ - if (isDescendant && !initialPosition.getInitialPositionInStream() - .equals(InitialPositionInStream.AT_TIMESTAMP)) { - newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - } else { + // If shard is a descendant, the leases for its ancestors were already created above. Open shards + // that are NOT descendants will not have leases yet, so we create them here. We will not create + // leases for open shards that ARE descendants yet - leases for these shards will be created upon + // SHARD_END of their parents. + if (!isDescendant) { + LOG.debug("ShardId " + shardId + " has no ancestors. Creating a lease."); + final KinesisClientLease newLease = KinesisShardSyncer.newKCLLease(shard); newLease.setCheckpoint(KinesisShardSyncer.convertToCheckpoint(initialPosition)); + LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint()); + shardIdToNewLeaseMap.put(shardId, newLease); + } else { + LOG.debug("ShardId " + shardId + " is a descendant whose ancestors should already have leases. " + + "Not creating a lease."); } - LOG.debug("Set checkpoint of " + newLease.getLeaseKey() + " to " + newLease.getCheckpoint()); - shardIdToNewLeaseMap.put(shardId, newLease); } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java index e7b6c285..7f53133b 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ExceptionThrowingLeaseManager.java @@ -225,7 +225,7 @@ class ExceptionThrowingLeaseManager implements ILeaseManager @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return false; + return leaseManager.listLeases().isEmpty(); } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index 48d71f6d..208d6448 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -18,6 +18,9 @@ import java.io.File; import java.io.IOException; import java.math.BigInteger; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -29,8 +32,10 @@ import java.util.stream.Stream; import com.amazonaws.services.dynamodbv2.AmazonDynamoDB; import com.amazonaws.services.dynamodbv2.local.embedded.DynamoDBEmbedded; +import com.amazonaws.services.kinesis.leases.impl.Lease; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.After; @@ -41,6 +46,7 @@ import org.junit.Test; import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.KinesisClientLibIOException; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ExceptionThrowingLeaseManager.ExceptionThrowingLeaseManagerMethods; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisShardSyncer.MemoizationContext; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.util.KinesisLocalFileDataCreator; @@ -58,10 +64,14 @@ import com.amazonaws.services.kinesis.model.Shard; import junit.framework.Assert; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * @@ -75,6 +85,7 @@ public class ShardSyncerTest { InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); private static final InitialPositionInStreamExtended INITIAL_POSITION_AT_TIMESTAMP = InitialPositionInStreamExtended.newInitialPositionAtTimestamp(new Date(1000L)); + private static String LEASE_OWNER = "leaseOwner"; private final boolean cleanupLeasesOfCompletedShards = true; private static final int EXPONENT = 128; AmazonDynamoDB ddbClient = DynamoDBEmbedded.create().amazonDynamoDB(); @@ -236,7 +247,7 @@ public class ShardSyncerTest { * All open and closed shards within stream's retention period should be sync'ed when lease table is empty. */ @Test - public final void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable() + public final void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable1() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { List shards = constructShardListForGraphA(); @@ -263,7 +274,7 @@ public class ShardSyncerTest { * We should only create leases for shards at LATEST when lease table is not empty. */ @Test - public final void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable() + public final void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable1() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { List shards = constructShardListForGraphA(); @@ -431,8 +442,11 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShard() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl(null, - Integer.MAX_VALUE, INITIAL_POSITION_TRIM_HORIZON); + Integer.MAX_VALUE, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); } /** @@ -446,12 +460,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithDeleteLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( - ExceptionThrowingLeaseManagerMethods.DELETELEASE, c, INITIAL_POSITION_TRIM_HORIZON); + ExceptionThrowingLeaseManagerMethods.DELETELEASE, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -468,12 +486,15 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithListLeasesExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( - ExceptionThrowingLeaseManagerMethods.LISTLEASES, c, INITIAL_POSITION_TRIM_HORIZON); + ExceptionThrowingLeaseManagerMethods.LISTLEASES, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -490,12 +511,15 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithCreateLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-0", "shardId-1", "shardId-2", "shardId-3", "shardId-4", "shardId-5" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 - int maxCallingCount = 5; + int maxCallingCount = 1; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( - ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, c,INITIAL_POSITION_TRIM_HORIZON); + ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -549,8 +573,11 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShard() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl(null, - Integer.MAX_VALUE, INITIAL_POSITION_AT_TIMESTAMP); + Integer.MAX_VALUE, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); } /** @@ -564,13 +591,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithDeleteLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.DELETELEASE, - c, INITIAL_POSITION_AT_TIMESTAMP); + c, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -587,13 +617,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithListLeasesExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 10; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.LISTLEASES, - c, INITIAL_POSITION_AT_TIMESTAMP); + c, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -610,13 +643,16 @@ public class ShardSyncerTest { public final void testCheckAndCreateLeasesForNewShardsAtTimestampAndClosedShardWithCreateLeaseExceptions() throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList( + "shardId-8", "shardId-4", "shardId-9", "shardId-10" + )); // Define the max calling count for lease manager methods. // From the Shard Graph, the max count of calling could be 10 int maxCallingCount = 5; for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, - c, INITIAL_POSITION_AT_TIMESTAMP); + c, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); // Need to clean up lease manager every time after calling KinesisShardSyncer leaseManager.deleteAll(); } @@ -626,7 +662,7 @@ public class ShardSyncerTest { private void testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods exceptionMethod, int exceptionTime, - InitialPositionInStreamExtended position) + InitialPositionInStreamExtended position, Set expectedLeaseKeysToCreate) throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, IOException { ExtendedSequenceNumber extendedSequenceNumber = @@ -634,35 +670,17 @@ public class ShardSyncerTest { List shards = constructShardListForGraphA(); File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); dataFile.deleteOnExit(); - IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); + final IKinesisProxy kinesisProxy = spy(new KinesisLocalFileProxy(dataFile.getAbsolutePath())); + when(kinesisProxy.getShardList()).thenReturn(shards); + when(kinesisProxy.getShardListWithFilter(any())).thenReturn(getFilteredShards(shards, position)); retryCheckAndCreateLeaseForNewShards(kinesisProxy, exceptionMethod, exceptionTime, position); List newLeases = leaseManager.listLeases(); Map expectedShardIdToCheckpointMap = new HashMap(); - for (int i = 0; i < 11; i++) { - expectedShardIdToCheckpointMap.put("shardId-" + i, extendedSequenceNumber); - } - Assert.assertEquals(expectedShardIdToCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease1 : newLeases) { - ExtendedSequenceNumber expectedCheckpoint = expectedShardIdToCheckpointMap.get(lease1.getLeaseKey()); - Assert.assertNotNull(expectedCheckpoint); - Assert.assertEquals(expectedCheckpoint, lease1.getCheckpoint()); - } + expectedLeaseKeysToCreate.forEach(l -> expectedShardIdToCheckpointMap.put(l, extendedSequenceNumber)); - KinesisClientLease closedShardLease = leaseManager.getLease("shardId-0"); - closedShardLease.setCheckpoint(ExtendedSequenceNumber.SHARD_END); - leaseManager.updateLease(closedShardLease); - expectedShardIdToCheckpointMap.remove(closedShardLease.getLeaseKey()); - KinesisClientLease childShardLease = leaseManager.getLease("shardId-6"); - childShardLease.setCheckpoint(new ExtendedSequenceNumber("34290")); - leaseManager.updateLease(childShardLease); - expectedShardIdToCheckpointMap.put(childShardLease.getLeaseKey(), new ExtendedSequenceNumber("34290")); - - retryCheckAndCreateLeaseForNewShards(kinesisProxy, exceptionMethod, exceptionTime, position); - - newLeases = leaseManager.listLeases(); Assert.assertEquals(expectedShardIdToCheckpointMap.size(), newLeases.size()); for (KinesisClientLease lease1 : newLeases) { ExtendedSequenceNumber expectedCheckpoint = expectedShardIdToCheckpointMap.get(lease1.getLeaseKey()); @@ -835,193 +853,333 @@ public class ShardSyncerTest { /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (3, 4, 5) + * Initial position: LATEST + * Leases to create: (2, 6) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeLatest1() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); - currentLeases.add(newLease("shardId-3")); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); - Map expectedShardIdCheckpointMap = + final Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.LATEST); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position Latest) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (4, 5, 7) + * Initial position: LATEST + * Leases to create: (6) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeLatest2() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - currentLeases.add(newLease("shardId-7")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_LATEST); - Map expectedShardIdCheckpointMap = + final Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (3, 4, 5) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: LATEST + * Leases to create: (3, 4, 9, 10) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeHorizon1() { + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange3() { List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); - currentLeases.add(newLease("shardId-3")); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) - * Current leases: (4, 5, 7) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: LATEST + * Leases to create: (8) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeHorizon2() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - currentLeases.add(newLease("shardId-7")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - Map expectedShardIdCheckpointMap = + final Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position TrimHorizon) - * For shard graph B (see the construct method doc for structure). + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is + * epochs): 0 1 2 3 - shards till + * / \ | \ / + * 4 5 1 6 - shards from epoch 103 - 205 + * / \ / \ | | + * 7 8 9 10 1 6 + * shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (9, 10) + * Initial position: LATEST + * Expected leases: (1, 6, 7, 8) + */ + @Test + public final void testDetermineNewLeasesToCreateSplitMergeLatestC_PartialHashRange5() { + final List shards = constructShardListForGraphC(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: LATEST + * Leases to create: empty set + */ + @Test + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedShardIdCheckpointMap = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public final void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", + "shardId-3", "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedShardIdCheckpointMap = Collections.emptyMap(); + + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: LATEST + * Expected leases: (4, 8, 9, 10) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (6) + * Initial position: LATEST + * Expected leases: (7) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_PartialHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (5) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: LATEST + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeLatestB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) * * Current leases: empty set + * Initial position: LATEST + * Expected leases: (9, 10) */ @Test - public final void testDetermineNewLeasesToCreateGraphBNoInitialLeasesTrim() { - List shards = constructShardListForGraphB(); - List currentLeases = new ArrayList(); - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); - Map expectedShardIdCheckpointMap = - new HashMap(); - for (int i = 0; i < 11; i++) { - String expectedShardId = "shardId-" + i; - expectedShardIdCheckpointMap.put(expectedShardId, ExtendedSequenceNumber.TRIM_HORIZON); - } - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeLatestB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.LATEST); + expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.LATEST); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_LATEST, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) + * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): * 0 1 2 3 4 5- shards till epoch 102 * \ / \ / | | @@ -1029,40 +1187,22 @@ public class ShardSyncerTest { * \ / | /\ * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeAtTimestamp1() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); - - currentLeases.add(newLease("shardId-3")); - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) + * Test CheckIfDescendantAndAddNewLeasesForAncestors * Shard structure (each level depicts a stream segment): * 0 1 2 3 4 5- shards till epoch 102 * \ / \ / | | @@ -1070,72 +1210,576 @@ public class ShardSyncerTest { * \ / | /\ * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (4, 5, 7) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) */ @Test - public final void testDetermineNewLeasesToCreateSplitMergeAtTimestamp2() { - List shards = constructShardListForGraphA(); - List currentLeases = new ArrayList(); - - currentLeases.add(newLease("shardId-4")); - currentLeases.add(newLease("shardId-5")); - currentLeases.add(newLease("shardId-7")); - - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - Map expectedShardIdCheckpointMap = new HashMap(); - expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-10", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-6", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); - expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); } /** - * Test CheckIfDescendantAndAddNewLeasesForAncestors (initial position AT_TIMESTAMP) - * For shard graph B (see the construct method doc for structure). - * Current leases: empty set + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: TRIM_HORIZON + * Expected leases: (3, 4, 5) */ @Test - public final void testDetermineNewLeasesToCreateGraphBNoInitialLeasesAtTimestamp() { - List shards = constructShardListForGraphB(); - List currentLeases = new ArrayList(); - final LeaseSynchronizer leaseSynchronizer = getLeaseSynchronizer(shards, currentLeases); - List newLeases = - shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); - Map expectedShardIdCheckpointMap = - new HashMap(); - for (int i = 0; i < shards.size(); i++) { - String expectedShardId = "shardId-" + i; - expectedShardIdCheckpointMap.put(expectedShardId, ExtendedSequenceNumber.AT_TIMESTAMP); - } - - Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); - for (KinesisClientLease lease : newLeases) { - Assert.assertTrue("Unexpected lease: " + lease, - expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); - Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); - } + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); } + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2, 3) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } - /* + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1, 2, 3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (6) + * Initial position: TRIM_HORIZON + * Expected leases: (7) + */ +// TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases +// @Test +// public void testDetermineNewLeasesToCreateSplitMergeHorizonB_PartialHashRange() { +// final List shards = constructShardListForGraphB(); +// final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); +// final Map expectedShardIdCheckpointMap = new HashMap<>(); +// expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.TRIM_HORIZON); +// testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); +// } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (5) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: TRIM_HORIZON + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedNoNewLeases); + } + + /** + * CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number)p + * + * Current leases: empty set + * Initial position: TRIM_HORIZON + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeHorizonB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.TRIM_HORIZON); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.TRIM_HORIZON); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_TRIM_HORIZON, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (3, 4, 5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange1() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-3", "shardId-4", "shardId-5"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange2() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-7"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (2, 6) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (3, 4, 5) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange3() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-2", "shardId-6"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 9, 10) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2, 3) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_PartialHashRange4() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-9", "shardId-10"); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (4, 5, 6, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRange() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 2, 3, 4, 5, 6, 7) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5", "shardId-6", "shardId-7"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: empty set + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1, 2, 3, 4, 5) */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_EmptyLeaseTable() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-2", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-3", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-4", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-5", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** + * Test CheckIfDescendantAndAddNewLeasesForAncestors + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Current leases: (0, 1, 4, 7, 9, 10) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampA_CompleteHashRangeAcrossDifferentEpochs() { + final List shards = constructShardListForGraphA(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-4", "shardId-7", + "shardId-9", "shardId-10"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (6) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (7) + */ +// TODO: Account for out-of-order lease creation in TRIM_HORIZON and AT_TIMESTAMP cases +// @Test +// public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_PartialHashRange() { +// final List shards = constructShardListForGraphB(); +// final List shardIdsOfCurrentLeases = Arrays.asList("shardId-6"); +// final Map expectedShardIdCheckpointMap = new HashMap<>(); +// expectedShardIdCheckpointMap.put("shardId-7", ExtendedSequenceNumber.AT_TIMESTAMP); +// testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); +// } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRange() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: (0, 1, 2, 3, 4, 5) + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: empty set + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_CompleteHashRangeWithoutGC() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Arrays.asList("shardId-0", "shardId-1", "shardId-2", "shardId-3", + "shardId-4", "shardId-5"); + final Map expectedNoNewLeases = Collections.emptyMap(); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedNoNewLeases); + } + + /** + * Shard structure (x-axis is epochs): + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) + * + * Current leases: empty set + * Initial position: AT_TIMESTAMP(1000) + * Expected leases: (0, 1) + */ + @Test + public void testDetermineNewLeasesToCreateSplitMergeAtTimestampB_EmptyLeaseTable() { + final List shards = constructShardListForGraphB(); + final List shardIdsOfCurrentLeases = Collections.emptyList(); + final Map expectedShardIdCheckpointMap = new HashMap<>(); + expectedShardIdCheckpointMap.put("shardId-0", ExtendedSequenceNumber.AT_TIMESTAMP); + expectedShardIdCheckpointMap.put("shardId-1", ExtendedSequenceNumber.AT_TIMESTAMP); + testCheckIfDescendantAndAddNewLeasesForAncestors(shards, shardIdsOfCurrentLeases, INITIAL_POSITION_AT_TIMESTAMP, expectedShardIdCheckpointMap); + } + + /** * Helper method to construct a shard list for graph A. Graph A is defined below. * Shard structure (y-axis is epochs): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 210 + * \ / | /\ + * 8 4 9 10 - shards from epoch 211 (open - no ending sequenceNumber) */ List constructShardListForGraphA() { List shards = new ArrayList(); @@ -1175,14 +1819,22 @@ public class ShardSyncerTest { return shards; } - /* + /** * Helper method to construct a shard list for graph B. Graph B is defined below. * Shard structure (x-axis is epochs): - * 0 3 6 9 - * \ / \ / \ / - * 2 5 8 - * / \ / \ / \ - * 1 4 7 10 + * 0 1 shards till epoch 1049 + * \ / + * 2 shards from epoch 1050 - 1099 + * / \ + * 3 4 shards from epoch 1100 - 1149 + * \ / + * 5 shards from epoch 1150 - 1199 + * / \ + * 6 7 shards from epoch 1200 - 1249 + * \ / + * 8 shards from epoch 1250 - 1299 + * / \ + * 9 10 shards from epoch 1300 (open - no ending sequence number) */ List constructShardListForGraphB() { List shards = new ArrayList(); @@ -1214,12 +1866,54 @@ public class ShardSyncerTest { return shards; } + /** + * Helper method to construct a shard list for graph C. Graph C is defined below. Shard structure (y-axis is + * epochs): 0 1 2 3 - shards till + * / \ | \ / + * 4 5 1 6 - shards from epoch 103 - 205 + * / \ / \ | | + * 7 8 9 10 1 6 + * shards from epoch 206 (open - no ending sequenceNumber) + */ + private List constructShardListForGraphC() { + final SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("11", "102"); + final SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("11", null); + final SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("103", null); + final SequenceNumberRange range3 = ShardObjectHelper.newSequenceNumberRange("103", "205"); + final SequenceNumberRange range4 = ShardObjectHelper.newSequenceNumberRange("206", null); + + return Arrays.asList( + ShardObjectHelper.newShard("shardId-0", null, null, range0, + ShardObjectHelper.newHashKeyRange("0", "399")), + ShardObjectHelper.newShard("shardId-1", null, null, range1, + ShardObjectHelper.newHashKeyRange("400", "499")), + ShardObjectHelper.newShard("shardId-2", null, null, range0, + ShardObjectHelper.newHashKeyRange("500", "599")), + ShardObjectHelper.newShard("shardId-3", null, null, range0, + ShardObjectHelper.newHashKeyRange("600", ShardObjectHelper.MAX_HASH_KEY)), + ShardObjectHelper.newShard("shardId-4", "shardId-0", null, range3, + ShardObjectHelper.newHashKeyRange("0", "199")), + ShardObjectHelper.newShard("shardId-5", "shardId-0", null, range3, + ShardObjectHelper.newHashKeyRange("200", "399")), + ShardObjectHelper.newShard("shardId-6", "shardId-2", "shardId-3", range2, + ShardObjectHelper.newHashKeyRange("500", ShardObjectHelper.MAX_HASH_KEY)), + ShardObjectHelper.newShard("shardId-7", "shardId-4", null, range4, + ShardObjectHelper.newHashKeyRange("0", "99")), + ShardObjectHelper.newShard("shardId-8", "shardId-4", null, range4, + ShardObjectHelper.newHashKeyRange("100", "199")), + ShardObjectHelper.newShard("shardId-9", "shardId-5", null, range4, + ShardObjectHelper.newHashKeyRange("200", "299")), + ShardObjectHelper.newShard("shardId-10", "shardId-5", null, range4, + ShardObjectHelper.newHashKeyRange("300", "399"))); + } + + /** * Test CheckIfDescendantAndAddNewLeasesForAncestors when shardId is null */ @Test public final void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() { - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST, null, null, @@ -1234,7 +1928,7 @@ public class ShardSyncerTest { public final void testCheckIfDescendantAndAddNewLeasesForAncestorsTrimmedShard() { String shardId = "shardId-trimmed"; Map kinesisShards = new HashMap(); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, null, kinesisShards, @@ -1253,7 +1947,7 @@ public class ShardSyncerTest { Set shardIdsOfCurrentLeases = new HashSet(); shardIdsOfCurrentLeases.add(shardId); Map newLeaseMap = new HashMap(); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertTrue(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, shardIdsOfCurrentLeases, kinesisShards, @@ -1280,7 +1974,7 @@ public class ShardSyncerTest { String shardId = "shardId-9-1"; kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null)); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, shardIdsOfCurrentLeases, kinesisShards, @@ -1309,7 +2003,7 @@ public class ShardSyncerTest { Shard shard = ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null); kinesisShards.put(shardId, shard); - Map memoizationContext = new HashMap<>(); + final MemoizationContext memoizationContext = new MemoizationContext(); Assert.assertTrue(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, shardIdsOfCurrentLeases, kinesisShards, @@ -1857,6 +2551,244 @@ public class ShardSyncerTest { shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); } + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: TRIM_HORIZON + * Leases to create: (0, 1, 2, 3, 4, 5) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithEmptyLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0", "shardId-1", "shardId-2", + "shardId-3", "shardId-4", "shardId-5")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: AT_TIMESTAMP(1000) + * Leases to create: (8, 4, 9, 10) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable1() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", + "shardId-10")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: AT_TIMESTAMP(200) + * Leases to create: (6, 7, 4, 5) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithEmptyLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-6", "shardId-7", "shardId-4", + "shardId-5")); + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPositionAtTimestamp(new Date(200L)); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Initial position: LATEST + * Leases to create: (8, 4, 9, 10) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtLatestWithEmptyLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-8", "shardId-4", "shardId-9", + "shardId-10")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate); + } + + private void testCheckAndCreateLeaseForShardsIfMissing(List shards,InitialPositionInStreamExtended initialPositionInStreamExtended, + Set expectedLeaseKeys) throws Exception { + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPositionInStreamExtended, expectedLeaseKeys, Collections.emptyList()); + } + + private void testCheckAndCreateLeaseForShardsIfMissing(List shards,InitialPositionInStreamExtended initialPositionInStreamExtended, + Set expectedLeaseKeys, List existingLeases) throws Exception { + final File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 0, "fileName"); + dataFile.deleteOnExit(); + final IKinesisProxy kinesisProxy = spy(new KinesisLocalFileProxy(dataFile.getAbsolutePath())); + when(kinesisProxy.getShardList()).thenReturn(shards); + when(kinesisProxy.getShardListWithFilter(any())).thenReturn(getFilteredShards(shards, initialPositionInStreamExtended)); + + // Populate existing leases + for (KinesisClientLease lease : existingLeases) { + leaseManager.createLeaseIfNotExists(lease); + } + + List oldLeases = leaseManager.listLeases(); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPositionInStreamExtended, + false, false); + List newLeases = leaseManager.listLeases(); + newLeases.removeAll(oldLeases); + + final Set newLeaseKeys = newLeases.stream().map(Lease::getLeaseKey).collect(Collectors.toSet()); + final Set newSequenceNumbers = newLeases.stream().map(KinesisClientLease::getCheckpoint).collect(Collectors.toSet()); + final Set expectedSequenceNumbers = new HashSet<>(Collections + .singletonList(new ExtendedSequenceNumber(initialPositionInStreamExtended.getInitialPositionInStream().name()))); + + assertThat(newLeases.size(), equalTo(expectedLeaseKeys.size())); + assertThat(newLeaseKeys, equalTo(expectedLeaseKeys)); + assertThat(newSequenceNumbers, equalTo(expectedSequenceNumbers)); + + dataFile.delete(); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: TRIM_HORIZON + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTrimHorizonWithPartialLeaseTable() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from TRIM_HORIZON. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.TRIM_HORIZON, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate, existingLeases); + } + + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: AT_TIMESTAMP(1000) + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable1() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from AT_TIMESTAMP. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeys= new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_AT_TIMESTAMP, expectedLeaseKeys, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: AT_TIMESTAMP(200) + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtTimestampWithPartialLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + final InitialPositionInStreamExtended initialPosition = InitialPositionInStreamExtended + .newInitialPositionAtTimestamp(new Date(200L)); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from AT_TIMESTAMP. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.AT_TIMESTAMP, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, initialPosition, expectedLeaseKeysToCreate, existingLeases); + } + + /* + * Shard structure (each level depicts a stream segment): + * 0 1 2 3 4 5- shards till epoch 102 + * \ / \ / | | + * 6 7 4 5- shards from epoch 103 - 205 + * \ / | /\ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * Missing leases: (0, 6, 8) + * Initial position: LATEST + * Leases to create: (0) + */ + @Test + public void testCheckAndCreateLeasesForNewShardsAtLatestWithPartialLeaseTable2() throws Exception { + final List shards = constructShardListForGraphA(); + // Leases for shard-0 and its descendants (shard-6, and shard-8) are missing. Expect lease sync to recover the + // lease for shard-0 when reading from LATEST. + final Set missingLeaseKeys = new HashSet<>(Arrays.asList("shardId-0", "shardId-6", "shardId-8")); + final List shardsWithLeases = shards.stream() + .filter(s -> !missingLeaseKeys.contains(s.getShardId())).collect(Collectors.toList()); + final List existingLeases = createLeasesFromShards(shardsWithLeases, ExtendedSequenceNumber.LATEST, LEASE_OWNER); + + final Set expectedLeaseKeysToCreate = new HashSet<>(Arrays.asList("shardId-0")); + testCheckAndCreateLeaseForShardsIfMissing(shards, INITIAL_POSITION_LATEST, expectedLeaseKeysToCreate, existingLeases); + } + + + + private List createLeasesFromShards(final List shards, final ExtendedSequenceNumber checkpoint, + final String leaseOwner) { + return shards.stream().map(shard -> { + final Set parentShardIds = new HashSet<>(); + if (StringUtils.isNotEmpty(shard.getParentShardId())) { + parentShardIds.add(shard.getParentShardId()); + } + if (StringUtils.isNotEmpty(shard.getAdjacentParentShardId())) { + parentShardIds.add(shard.getAdjacentParentShardId()); + } + + final KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey(shard.getShardId()); + lease.setLeaseOwner(leaseOwner); + lease.setLeaseCounter(0L); + lease.setLastCounterIncrementNanos(0L); + lease.setCheckpoint(checkpoint); + lease.setOwnerSwitchesSinceCheckpoint(0L); + lease.setParentShardIds(parentShardIds); + + return lease; + }).collect(Collectors.toList()); + } /** * Helper method. * @@ -1870,6 +2802,33 @@ public class ShardSyncerTest { return lease; } + /** + * Helper method to test CheckIfDescendantAndAddNewLeasesForAncestors and verify new leases created with an expected result. + * @param shards + * @param shardIdsOfCurrentLeases + * @param checkpoint + * @param expectedShardIdCheckpointMap + */ + private void testCheckIfDescendantAndAddNewLeasesForAncestors(List shards, List shardIdsOfCurrentLeases, + InitialPositionInStreamExtended checkpoint, Map expectedShardIdCheckpointMap) { + final List currentLeases = shardIdsOfCurrentLeases.stream() + .map(shardId -> newLease(shardId)).collect(Collectors.toList()); + final Map shardIdToShardMap = KinesisShardSyncer.constructShardIdToShardMap(shards); + final Map> shardIdToChildShardIdsMap = + KinesisShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); + + final LeaseSynchronizer leaseSynchronizer = new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap); + final List newLeases = + shardSyncer.determineNewLeasesToCreate(leaseSynchronizer, shards, currentLeases, checkpoint); + + Assert.assertEquals(expectedShardIdCheckpointMap.size(), newLeases.size()); + for (KinesisClientLease lease : newLeases) { + Assert.assertTrue("Unexpected lease: " + lease, + expectedShardIdCheckpointMap.containsKey(lease.getLeaseKey())); + Assert.assertEquals(expectedShardIdCheckpointMap.get(lease.getLeaseKey()), lease.getCheckpoint()); + } + } + /** * Helper method to get appropriate LeaseSynchronizer based on available shards and current leases. If there are * no current leases (empty lease table case), return EmptyLeaseTableSynchronizer. Else, return @@ -1891,4 +2850,33 @@ public class ShardSyncerTest { return new NonEmptyLeaseTableSynchronizer(shardIdToShardMap, shardIdToChildShardIdsMap); } + + /** + * Helper method to mimic behavior of Kinesis ListShardsWithFilter calls. + */ + private static List getFilteredShards(List shards, InitialPositionInStreamExtended initialPosition) { + switch (initialPosition.getInitialPositionInStream()) { + case LATEST: + return shards.stream() + .filter(s -> s.getSequenceNumberRange().getEndingSequenceNumber() == null) + .collect(Collectors.toList()); + case TRIM_HORIZON: + String minSeqNum = shards.stream() + .min(Comparator.comparingLong(s -> Long.parseLong(s.getSequenceNumberRange().getStartingSequenceNumber()))) + .map(s -> s.getSequenceNumberRange().getStartingSequenceNumber()) + .orElseThrow(RuntimeException::new); + return shards.stream() + .filter(s -> s.getSequenceNumberRange().getStartingSequenceNumber().equals(minSeqNum)) + .collect(Collectors.toList()); + case AT_TIMESTAMP: + return shards.stream() + .filter(s -> new Date(Long.parseLong(s.getSequenceNumberRange().getStartingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) <= 0) + .filter(s -> s.getSequenceNumberRange().getEndingSequenceNumber() == null || + new Date(Long.parseLong(s.getSequenceNumberRange().getEndingSequenceNumber())) + .compareTo(initialPosition.getTimestamp()) > 0) + .collect(Collectors.toList()); + } + throw new RuntimeException("Unsupported initial position " + initialPosition); + } } From 0a5724c3320714d31dc7d5bf93f279d594c6a2ba Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 13 May 2020 22:09:29 -0700 Subject: [PATCH 09/43] KCL 1.x ShardEnd Shard Sync and Lease table ChildShard persistence --- .../lib/worker/ConsumerStates.java | 3 +- .../lib/worker/KinesisDataFetcher.java | 48 +++++- .../lib/worker/KinesisShardSyncer.java | 24 +++ .../lib/worker/PrefetchGetRecordsCache.java | 2 +- .../clientlibrary/lib/worker/ProcessTask.java | 6 +- .../lib/worker/ShardConsumer.java | 11 ++ .../lib/worker/ShutdownTask.java | 93 ++++++++---- .../clientlibrary/lib/worker/TaskResult.java | 23 +++ .../leases/impl/KinesisClientLease.java | 13 +- .../impl/KinesisClientLeaseSerializer.java | 11 +- .../lib/worker/KinesisDataFetcherTest.java | 46 ++++-- ...refetchGetRecordsCacheIntegrationTest.java | 7 +- .../worker/PrefetchGetRecordsCacheTest.java | 5 +- .../lib/worker/ShardConsumerTest.java | 31 +++- .../lib/worker/ShutdownTaskTest.java | 137 ++++++++++-------- .../proxies/KinesisLocalFileProxy.java | 21 +++ .../impl/KinesisClientLeaseBuilder.java | 5 +- 17 files changed, 349 insertions(+), 137 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index 47b1239f..fc3400e8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -530,7 +530,8 @@ class ConsumerStates { consumer.isIgnoreUnexpectedChildShards(), consumer.getLeaseCoordinator(), consumer.getTaskBackoffTimeMillis(), - consumer.getGetRecordsCache(), consumer.getShardSyncer(), consumer.getShardSyncStrategy()); + consumer.getGetRecordsCache(), consumer.getShardSyncer(), + consumer.getShardSyncStrategy(), consumer.getChildShards()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java index e425e070..c716afa1 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java @@ -16,7 +16,12 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.Collections; import java.util.Date; +import java.util.List; +import java.util.Set; +import com.amazonaws.SdkClientException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.model.ChildShard; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -47,6 +52,7 @@ class KinesisDataFetcher { private boolean isInitialized; private String lastKnownSequenceNumber; private InitialPositionInStreamExtended initialPositionInStream; + private List childShards = Collections.emptyList(); /** * @@ -85,8 +91,11 @@ class KinesisDataFetcher { final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() { @Override public GetRecordsResult getResult() { - return new GetRecordsResult().withMillisBehindLatest(null).withRecords(Collections.emptyList()) - .withNextShardIterator(null); + return new GetRecordsResult() + .withMillisBehindLatest(null) + .withRecords(Collections.emptyList()) + .withNextShardIterator(null) + .withChildShards(Collections.emptyList()); } @Override @@ -113,12 +122,20 @@ class KinesisDataFetcher { @Override public GetRecordsResult accept() { + if (!isValidResult(result)) { + // Throwing SDK exception when the GetRecords result is not valid. This will allow PrefetchGetRecordsCache to retry the GetRecords call. + throw new SdkClientException("Shard " + shardId +": GetRecordsResult is not valid. NextShardIterator: " + result.getNextShardIterator() + + ". ChildShards: " + result.getChildShards()); + } nextIterator = result.getNextShardIterator(); if (!CollectionUtils.isNullOrEmpty(result.getRecords())) { lastKnownSequenceNumber = Iterables.getLast(result.getRecords()).getSequenceNumber(); } if (nextIterator == null) { - LOG.info("Reached shard end: nextIterator is null in AdvancingResult.accept for shard " + shardId); + LOG.info("Reached shard end: nextIterator is null in AdvancingResult.accept for shard " + shardId + ". childShards: " + result.getChildShards()); + if (!CollectionUtils.isNullOrEmpty(result.getChildShards())) { + childShards = result.getChildShards(); + } isShardEndReached = true; } return getResult(); @@ -130,6 +147,23 @@ class KinesisDataFetcher { } } + private boolean isValidResult(GetRecordsResult getRecordsResult) { + // GetRecords result should contain childShard information. There are two valid combination for the nextShardIterator and childShards + // If the GetRecords call does not reach the shard end, getRecords result should contain a non-null nextShardIterator and an empty list of childShards. + // If the GetRecords call reaches the shard end, getRecords result should contain a null nextShardIterator and a non-empty list of childShards. + // All other combinations are invalid and indicating an issue with GetRecords result from Kinesis service. + if (getRecordsResult.getNextShardIterator() == null && CollectionUtils.isNullOrEmpty(getRecordsResult.getChildShards()) || + getRecordsResult.getNextShardIterator() != null && !CollectionUtils.isNullOrEmpty(getRecordsResult.getChildShards())) { + return false; + } + for (ChildShard childShard : getRecordsResult.getChildShards()) { + if (CollectionUtils.isNullOrEmpty(childShard.getParentShards())) { + return false; + } + } + return true; + } + /** * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number. * @param initialCheckpoint Current checkpoint sequence number for this shard. @@ -141,8 +175,7 @@ class KinesisDataFetcher { isInitialized = true; } - public void initialize(ExtendedSequenceNumber initialCheckpoint, - InitialPositionInStreamExtended initialPositionInStream) { + public void initialize(ExtendedSequenceNumber initialCheckpoint, InitialPositionInStreamExtended initialPositionInStream) { LOG.info("Initializing shard " + shardId + " with " + initialCheckpoint.getSequenceNumber()); advanceIteratorTo(initialCheckpoint.getSequenceNumber(), initialPositionInStream); isInitialized = true; @@ -171,6 +204,7 @@ class KinesisDataFetcher { if (nextIterator == null) { LOG.info("Reached shard end: cannot advance iterator for shard " + shardId); isShardEndReached = true; + // TODO: transition to ShuttingDown state on shardend instead to shutdown state for enqueueing this for cleanup } this.lastKnownSequenceNumber = sequenceNumber; this.initialPositionInStream = initialPositionInStream; @@ -248,6 +282,10 @@ class KinesisDataFetcher { return isShardEndReached; } + protected List getChildShards() { + return childShards; + } + /** Note: This method has package level access for testing purposes. * @return nextIterator */ diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index 21890663..9811e35e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -28,6 +28,7 @@ import java.util.Set; import com.amazonaws.services.kinesis.leases.impl.Lease; import com.amazonaws.services.kinesis.leases.impl.LeaseManager; +import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; import com.amazonaws.util.CollectionUtils; @@ -735,6 +736,29 @@ class KinesisShardSyncer implements ShardSyncer { return newLease; } + /** + * Helper method to create a new KinesisClientLease POJO for a ChildShard. + * Note: Package level access only for testing purposes + * + * @param childShard + * @return + */ + static KinesisClientLease newKCLLeaseForChildShard(ChildShard childShard) throws InvalidStateException { + final KinesisClientLease newLease = new KinesisClientLease(); + newLease.setLeaseKey(childShard.getShardId()); + final List parentShardIds = new ArrayList<>(); + if (!CollectionUtils.isNullOrEmpty(childShard.getParentShards())) { + parentShardIds.addAll(childShard.getParentShards()); + } else { + throw new InvalidStateException("Unable to populate new lease for child shard " + childShard.getShardId() + + " because parent shards cannot be found."); + } + newLease.setParentShardIds(parentShardIds); + newLease.setOwnerSwitchesSinceCheckpoint(0L); + newLease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); + return newLease; + } + /** * Helper method to construct a shardId->Shard map for the specified list of shards. * diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java index 8173a479..a4cf74d8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java @@ -129,6 +129,7 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { try { result = getRecordsResultQueue.take().withCacheExitTime(Instant.now()); prefetchCounters.removed(result); + log.info("Shard " + shardId + ": Number of records remaining in queue is " + getRecordsResultQueue.size()); } catch (InterruptedException e) { log.error("Interrupted while getting records from the cache", e); } @@ -177,7 +178,6 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { MetricsHelper.getMetricsScope().addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.Count, MetricsLevel.SUMMARY); - dataFetcher.restartIterator(); } catch (SdkClientException e) { log.error("Exception thrown while fetching records from Kinesis", e); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java index b578fbb0..cd543e23 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java @@ -152,8 +152,8 @@ class ProcessTask implements ITask { try { if (dataFetcher.isShardEndReached()) { - LOG.info("Reached end of shard " + shardInfo.getShardId()); - return new TaskResult(null, true); + LOG.info("Reached end of shard " + shardInfo.getShardId() + ". Found childShards: " + dataFetcher.getChildShards()); + return new TaskResult(null, true, dataFetcher.getChildShards()); } final ProcessRecordsInput processRecordsInput = getRecordsResult(); @@ -353,7 +353,7 @@ class ProcessTask implements ITask { * recordProcessorCheckpointer). */ dataFetcher.advanceIteratorTo(recordProcessorCheckpointer.getLargestPermittedCheckpointValue() - .getSequenceNumber(), streamConfig.getInitialPositionInStream()); + .getSequenceNumber(), streamConfig.getInitialPositionInStream()); // Try a second time - if we fail this time, expose the failure. try { diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index a30412ce..f5513d3e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -15,11 +15,14 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; +import com.amazonaws.services.kinesis.model.ChildShard; +import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -66,6 +69,9 @@ class ShardConsumer { private Future future; private ShardSyncStrategy shardSyncStrategy; + @Getter + private List childShards; + @Getter private final GetRecordsCache getRecordsCache; @@ -321,6 +327,10 @@ class ShardConsumer { TaskResult result = future.get(); if (result.getException() == null) { if (result.isShardEndReached()) { + if (!CollectionUtils.isNullOrEmpty(result.getChildShards())) { + childShards = result.getChildShards(); + LOG.info("Shard " + shardInfo.getShardId() + ": Setting childShards in ShardConsumer: " + childShards); + } return TaskOutcome.END_OF_SHARD; } return TaskOutcome.SUCCESSFUL; @@ -420,6 +430,7 @@ class ShardConsumer { void updateState(TaskOutcome taskOutcome) { if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.TERMINATE); + LOG.info("Shard " + shardInfo.getShardId() + ": Mark for shutdown with reason TERMINATE"); } if (isShutdownRequested() && taskOutcome != TaskOutcome.FAILURE) { currentState = currentState.shutdownTransition(shutdownReason); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index a9ff5080..71cf3b9d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -14,9 +14,11 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; -import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardClosureVerificationResponse; -import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; -import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.model.ChildShard; +import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,6 +32,9 @@ import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.google.common.annotations.VisibleForTesting; import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; /** * Task for invoking the RecordProcessor shutdown() callback. @@ -54,6 +59,7 @@ class ShutdownTask implements ITask { private final GetRecordsCache getRecordsCache; private final ShardSyncer shardSyncer; private final ShardSyncStrategy shardSyncStrategy; + private final List childShards; /** * Constructor. @@ -69,7 +75,8 @@ class ShutdownTask implements ITask { boolean ignoreUnexpectedChildShards, KinesisClientLibLeaseCoordinator leaseCoordinator, long backoffTimeMillis, - GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) { + GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, + ShardSyncStrategy shardSyncStrategy, List childShards) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -83,6 +90,7 @@ class ShutdownTask implements ITask { this.getRecordsCache = getRecordsCache; this.shardSyncer = shardSyncer; this.shardSyncStrategy = shardSyncStrategy; + this.childShards = childShards; } /* @@ -97,29 +105,39 @@ class ShutdownTask implements ITask { boolean applicationException = false; try { + LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: " + + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); ShutdownReason localReason = reason; - List latestShards = null; /* * Revalidate if the current shard is closed before shutting down the shard consumer with reason SHARD_END * If current shard is not closed, shut down the shard consumer with reason LEASE_LOST that allows active * workers to contend for the lease of this shard. */ if(localReason == ShutdownReason.TERMINATE) { - ShardClosureVerificationResponse shardClosureVerificationResponse = kinesisProxy.verifyShardClosure(shardInfo.getShardId()); - if (shardClosureVerificationResponse instanceof ShardListWrappingShardClosureVerificationResponse) { - latestShards = ((ShardListWrappingShardClosureVerificationResponse)shardClosureVerificationResponse).getLatestShards(); - } - - // If shard in context is not closed yet we should shut down the ShardConsumer with Zombie state - // which avoids checkpoint-ing with SHARD_END sequence number. - if(!shardClosureVerificationResponse.isShardClosed()) { + // Create new lease for the child shards if they don't exist. + // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. + // This would happen when KinesisDataFetcher catches ResourceNotFound exception. + // In this case, KinesisDataFetcher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. + // This scenario could happen when customer deletes the stream while leaving the KCL application running. + try { + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateCurrentLeaseWithChildShards(); + } else { + LOG.warn("Shard " + shardInfo.getShardId() + + ": Shutting down consumer with SHARD_END reason without creating leases for child shards."); + } + } catch (InvalidStateException e) { + // If invalidStateException happens, it indicates we are missing childShard related information. + // In this scenario, we should shutdown the shardConsumer with ZOMBIE reason to allow other worker to take the lease and retry getting + // childShard information in the processTask. localReason = ShutdownReason.ZOMBIE; dropLease(); - LOG.info("Forcing the lease to be lost before shutting down the consumer for Shard: " + shardInfo.getShardId()); + LOG.warn("Shard " + shardInfo.getShardId() + ": Exception happened while shutting down shardConsumer with TERMINATE reason. " + + "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); } } - // If we reached end of the shard, set sequence number to SHARD_END. if (localReason == ShutdownReason.TERMINATE) { recordProcessorCheckpointer.setSequenceNumberAtShardEnd( @@ -127,8 +145,6 @@ class ShutdownTask implements ITask { recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); } - LOG.debug("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken " - + shardInfo.getConcurrencyToken() + ". Shutdown reason: " + localReason); final ShutdownInput shutdownInput = new ShutdownInput() .withShutdownReason(localReason) .withCheckpointer(recordProcessorCheckpointer); @@ -156,18 +172,6 @@ class ShutdownTask implements ITask { MetricsLevel.SUMMARY); } - if (localReason == ShutdownReason.TERMINATE) { - LOG.debug("Looking for child shards of shard " + shardInfo.getShardId()); - // create leases for the child shards - TaskResult result = shardSyncStrategy.onShardConsumerShutDown(latestShards); - if (result.getException() != null) { - LOG.debug("Exception while trying to sync shards on the shutdown of shard: " + shardInfo - .getShardId()); - throw result.getException(); - } - LOG.debug("Finished checking for child shards of shard " + shardInfo.getShardId()); - } - return new TaskResult(null); } catch (Exception e) { if (applicationException) { @@ -187,6 +191,33 @@ class ShutdownTask implements ITask { return new TaskResult(exception); } + private void createLeasesForChildShardsIfNotExist() throws InvalidStateException, DependencyException, ProvisionedThroughputException { + for (ChildShard childShard : childShards) { + final String leaseKey = childShard.getShardId(); + if (leaseCoordinator.getLeaseManager().getLease(leaseKey) == null) { + final KinesisClientLease leaseToCreate = KinesisShardSyncer.newKCLLeaseForChildShard(childShard); + leaseCoordinator.getLeaseManager().createLeaseIfNotExists(leaseToCreate); + LOG.info("Shard " + shardInfo.getShardId() + " : Created child shard lease: " + leaseToCreate.getLeaseKey()); + } + } + } + + private void updateCurrentLeaseWithChildShards() throws DependencyException, InvalidStateException, ProvisionedThroughputException { + final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + if (currentLease == null) { + throw new InvalidStateException("Failed to retrieve current lease for shard " + shardInfo.getShardId()); + } + final Set childShardIds = childShards.stream().map(ChildShard::getShardId).collect(Collectors.toSet()); + + currentLease.setChildShardIds(childShardIds); + final boolean updateResult = leaseCoordinator.updateLease(currentLease, UUID.fromString(shardInfo.getConcurrencyToken())); + if (!updateResult) { + throw new InvalidStateException("Failed to update parent lease with child shard information for shard " + shardInfo.getShardId()); + } + LOG.info("Shard " + shardInfo.getShardId() + ": Updated current lease with child shard information: " + currentLease.getLeaseKey()); + } + + /* * (non-Javadoc) * @@ -204,6 +235,10 @@ class ShutdownTask implements ITask { private void dropLease() { KinesisClientLease lease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + if (lease == null) { + LOG.warn("Shard " + shardInfo.getShardId() + ": Lease already dropped. Will shutdown the shardConsumer directly."); + return; + } leaseCoordinator.dropLease(lease); LOG.warn("Dropped lease for shutting down ShardConsumer: " + lease.getLeaseKey()); } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java index bc68d292..70109b86 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/TaskResult.java @@ -14,6 +14,10 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import com.amazonaws.services.kinesis.model.ChildShard; + +import java.util.List; + /** * Used to capture information from a task that we want to communicate back to the higher layer. * E.g. exception thrown when executing the task, if we reach end of a shard. @@ -26,6 +30,9 @@ class TaskResult { // Any exception caught while executing the task. private Exception exception; + // List of childShards of the current shard. This field is only required for the task result when we reach end of a shard. + private List childShards; + /** * @return the shardEndReached */ @@ -33,6 +40,11 @@ class TaskResult { return shardEndReached; } + /** + * @return the list of childShards. + */ + protected List getChildShards() { return childShards; } + /** * @param shardEndReached the shardEndReached to set */ @@ -40,6 +52,11 @@ class TaskResult { this.shardEndReached = shardEndReached; } + /** + * @param childShards the list of childShards to set + */ + protected void setChildShards(List childShards) { this.childShards = childShards; } + /** * @return the exception */ @@ -70,4 +87,10 @@ class TaskResult { this.shardEndReached = isShardEndReached; } + TaskResult(Exception e, boolean isShardEndReached, List childShards) { + this.exception = e; + this.shardEndReached = isShardEndReached; + this.childShards = childShards; + } + } diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java index ae58fb10..259b4c2f 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java @@ -30,9 +30,10 @@ public class KinesisClientLease extends Lease { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet(); - private Set childShardIds = new HashSet(); + private Set childShardIds = new HashSet<>(); private HashKeyRangeForLease hashKeyRangeForLease; + public KinesisClientLease() { } @@ -43,7 +44,7 @@ public class KinesisClientLease extends Lease { this.pendingCheckpoint = other.getPendingCheckpoint(); this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint(); this.parentShardIds.addAll(other.getParentShardIds()); - this.childShardIds = other.getChildShardIds(); + this.childShardIds.addAll(other.getChildShardIds()); this.hashKeyRangeForLease = other.getHashKeyRange(); } @@ -76,6 +77,7 @@ public class KinesisClientLease extends Lease { setCheckpoint(casted.checkpoint); setPendingCheckpoint(casted.pendingCheckpoint); setParentShardIds(casted.parentShardIds); + setChildShardIds(casted.childShardIds); } /** @@ -108,7 +110,7 @@ public class KinesisClientLease extends Lease { } /** - * @return shardIds that are the children of this lease. Used for resharding. + * @return shardIds for the child shards of the current shard. Used for resharding. */ public Set getChildShardIds() { return new HashSet(childShardIds); @@ -170,9 +172,6 @@ public class KinesisClientLease extends Lease { * @param childShardIds may not be null */ public void setChildShardIds(Collection childShardIds) { - verifyNotNull(childShardIds, "childShardIds should not be null"); - - this.childShardIds.clear(); this.childShardIds.addAll(childShardIds); } @@ -186,7 +185,7 @@ public class KinesisClientLease extends Lease { this.hashKeyRangeForLease = hashKeyRangeForLease; } - + private void verifyNotNull(Object object, String message) { if (object == null) { throw new IllegalArgumentException(message); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 6bf9bc58..0b9271be 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -43,7 +43,7 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer recordsA = new ArrayList(); outputA.setRecords(recordsA); + outputA.setNextShardIterator("nextShardIteratorA"); + outputA.setChildShards(Collections.emptyList()); GetRecordsResult outputB = new GetRecordsResult(); List recordsB = new ArrayList(); outputB.setRecords(recordsB); + outputB.setNextShardIterator("nextShardIteratorB"); + outputB.setChildShards(Collections.emptyList()); when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqA)).thenReturn(iteratorA); when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqB)).thenReturn(iteratorB); @@ -166,7 +171,7 @@ public class KinesisDataFetcherTest { } @Test - public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() { + public void testadvanceIteratorToTrimHorizonLatestAndAtTimestamp() throws Exception{ IKinesisProxy kinesis = mock(IKinesisProxy.class); KinesisDataFetcher fetcher = new KinesisDataFetcher(kinesis, SHARD_INFO); @@ -189,7 +194,7 @@ public class KinesisDataFetcherTest { } @Test - public void testGetRecordsWithResourceNotFoundException() { + public void testGetRecordsWithResourceNotFoundException() throws Exception { // Set up arguments used by proxy String nextIterator = "TestShardIterator"; int maxRecords = 100; @@ -211,11 +216,12 @@ public class KinesisDataFetcherTest { } @Test - public void testNonNullGetRecords() { + public void testNonNullGetRecords() throws Exception { String nextIterator = "TestIterator"; int maxRecords = 100; KinesisProxy mockProxy = mock(KinesisProxy.class); + when(mockProxy.getIterator(anyString(), anyString())).thenReturn("targetIterator"); doThrow(new ResourceNotFoundException("Test Exception")).when(mockProxy).get(nextIterator, maxRecords); KinesisDataFetcher dataFetcher = new KinesisDataFetcher(mockProxy, SHARD_INFO); @@ -232,17 +238,25 @@ public class KinesisDataFetcherTest { final String NEXT_ITERATOR_ONE = "NextIteratorOne"; final String NEXT_ITERATOR_TWO = "NextIteratorTwo"; when(kinesisProxy.getIterator(anyString(), anyString())).thenReturn(INITIAL_ITERATOR); - GetRecordsResult iteratorOneResults = mock(GetRecordsResult.class); - when(iteratorOneResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_ONE); + + GetRecordsResult iteratorOneResults = new GetRecordsResult(); + iteratorOneResults.setNextShardIterator(NEXT_ITERATOR_ONE); + iteratorOneResults.setChildShards(Collections.emptyList()); when(kinesisProxy.get(eq(INITIAL_ITERATOR), anyInt())).thenReturn(iteratorOneResults); - GetRecordsResult iteratorTwoResults = mock(GetRecordsResult.class); + GetRecordsResult iteratorTwoResults = new GetRecordsResult(); + iteratorTwoResults.setNextShardIterator(NEXT_ITERATOR_TWO); + iteratorTwoResults.setChildShards(Collections.emptyList()); when(kinesisProxy.get(eq(NEXT_ITERATOR_ONE), anyInt())).thenReturn(iteratorTwoResults); - when(iteratorTwoResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_TWO); - GetRecordsResult finalResult = mock(GetRecordsResult.class); + GetRecordsResult finalResult = new GetRecordsResult(); + finalResult.setNextShardIterator(null); + List childShards = new ArrayList<>(); + ChildShard childShard = new ChildShard(); + childShard.setParentShards(Collections.singletonList("parentShardId")); + childShards.add(childShard); + finalResult.setChildShards(childShards); when(kinesisProxy.get(eq(NEXT_ITERATOR_TWO), anyInt())).thenReturn(finalResult); - when(finalResult.getNextShardIterator()).thenReturn(null); KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); dataFetcher.initialize("TRIM_HORIZON", @@ -276,13 +290,14 @@ public class KinesisDataFetcherTest { } @Test - public void testRestartIterator() { + public void testRestartIterator() throws Exception{ GetRecordsResult getRecordsResult = mock(GetRecordsResult.class); - GetRecordsResult restartGetRecordsResult = new GetRecordsResult(); + GetRecordsResult restartGetRecordsResult = mock(GetRecordsResult.class); Record record = mock(Record.class); final String initialIterator = "InitialIterator"; final String nextShardIterator = "NextShardIterator"; final String restartShardIterator = "RestartIterator"; + final String restartNextShardIterator = "RestartNextIterator"; final String sequenceNumber = "SequenceNumber"; final String iteratorType = "AT_SEQUENCE_NUMBER"; KinesisProxy kinesisProxy = mock(KinesisProxy.class); @@ -292,6 +307,7 @@ public class KinesisDataFetcherTest { when(kinesisProxy.get(eq(initialIterator), eq(10))).thenReturn(getRecordsResult); when(getRecordsResult.getRecords()).thenReturn(Collections.singletonList(record)); when(getRecordsResult.getNextShardIterator()).thenReturn(nextShardIterator); + when(getRecordsResult.getChildShards()).thenReturn(Collections.emptyList()); when(record.getSequenceNumber()).thenReturn(sequenceNumber); fetcher.initialize(InitialPositionInStream.LATEST.toString(), INITIAL_POSITION_LATEST); @@ -300,6 +316,8 @@ public class KinesisDataFetcherTest { verify(kinesisProxy).get(eq(initialIterator), eq(10)); when(kinesisProxy.getIterator(eq(SHARD_ID), eq(iteratorType), eq(sequenceNumber))).thenReturn(restartShardIterator); + when(restartGetRecordsResult.getNextShardIterator()).thenReturn(restartNextShardIterator); + when(restartGetRecordsResult.getChildShards()).thenReturn(Collections.emptyList()); when(kinesisProxy.get(eq(restartShardIterator), eq(10))).thenReturn(restartGetRecordsResult); fetcher.restartIterator(); @@ -309,7 +327,7 @@ public class KinesisDataFetcherTest { } @Test (expected = IllegalStateException.class) - public void testRestartIteratorNotInitialized() { + public void testRestartIteratorNotInitialized() throws Exception { KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); dataFetcher.restartIterator(); } @@ -354,6 +372,8 @@ public class KinesisDataFetcherTest { List expectedRecords = new ArrayList(); GetRecordsResult response = new GetRecordsResult(); response.setRecords(expectedRecords); + response.setNextShardIterator("testNextShardIterator"); + response.setChildShards(Collections.emptyList()); when(kinesis.getIterator(SHARD_ID, initialPositionInStream.getTimestamp())).thenReturn(iterator); when(kinesis.getIterator(SHARD_ID, AT_SEQUENCE_NUMBER, seqNo)).thenReturn(iterator); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java index f77d3a9c..8043e0bf 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java @@ -29,6 +29,7 @@ import static org.mockito.Mockito.when; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -74,6 +75,8 @@ public class PrefetchGetRecordsCacheIntegrationTest { private IKinesisProxy proxy; @Mock private ShardInfo shardInfo; + @Mock + private KinesisClientLibLeaseCoordinator leaseCoordinator; @Before public void setup() { @@ -171,7 +174,7 @@ public class PrefetchGetRecordsCacheIntegrationTest { } @Test - public void testExpiredIteratorException() { + public void testExpiredIteratorException() throws Exception { when(dataFetcher.getRecords(eq(MAX_RECORDS_PER_CALL))).thenAnswer(new Answer() { @Override public DataFetcherResult answer(final InvocationOnMock invocationOnMock) throws Throwable { @@ -215,6 +218,8 @@ public class PrefetchGetRecordsCacheIntegrationTest { GetRecordsResult getRecordsResult = new GetRecordsResult(); getRecordsResult.setRecords(new ArrayList<>(records)); getRecordsResult.setMillisBehindLatest(1000L); + getRecordsResult.setNextShardIterator("testNextShardIterator"); + getRecordsResult.setChildShards(Collections.emptyList()); return new AdvancingResult(getRecordsResult); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java index a4336aad..b6d7769e 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java @@ -31,6 +31,7 @@ import static org.mockito.Mockito.when; import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -98,6 +99,8 @@ public class PrefetchGetRecordsCacheTest { when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResult); when(getRecordsResult.getRecords()).thenReturn(records); + when(getRecordsResult.getNextShardIterator()).thenReturn("testNextShardIterator"); + when(getRecordsResult.getChildShards()).thenReturn(Collections.emptyList()); } @Test @@ -203,7 +206,7 @@ public class PrefetchGetRecordsCacheTest { } @Test - public void testExpiredIteratorException() { + public void testExpiredIteratorException() throws Exception{ getRecordsCache.start(); when(getRecordsRetrievalStrategy.getRecords(MAX_RECORDS_PER_CALL)).thenThrow(ExpiredIteratorException.class).thenReturn(getRecordsResult); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 1cf86c4f..f040c6a6 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -39,6 +39,7 @@ import java.io.File; import java.math.BigInteger; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Date; import java.util.List; import java.util.ListIterator; @@ -245,7 +246,7 @@ public class ShardConsumerTest { @SuppressWarnings("unchecked") @Test public final void testRecordProcessorThrowable() throws Exception { - ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + ShardInfo shardInfo = new ShardInfo("s-0-0", UUID.randomUUID().toString(), null, ExtendedSequenceNumber.TRIM_HORIZON); StreamConfig streamConfig = new StreamConfig(streamProxy, 1, @@ -271,6 +272,7 @@ public class ShardConsumerTest { final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = null; + when(streamProxy.getIterator(anyString(), anyString(), anyString())).thenReturn("startingIterator"); when(leaseManager.getLease(anyString())).thenReturn(null); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(checkpoint.getCheckpointObject(anyString())).thenReturn( @@ -538,7 +540,7 @@ public class ShardConsumerTest { int numRecs = 10; BigInteger startSeqNum = BigInteger.ONE; String streamShardId = "kinesis-0-0"; - String testConcurrencyToken = "testToken"; + String testConcurrencyToken = UUID.randomUUID().toString(); List shardList = KinesisLocalFileDataCreator.createShardList(1, "kinesis-0-", startSeqNum); // Close the shard so that shutdown is called with reason terminate shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( @@ -606,8 +608,7 @@ public class ShardConsumerTest { shardSyncer, shardSyncStrategy); - when(shardSyncStrategy.onShardConsumerShutDown(shardList)).thenReturn(new TaskResult(null)); - + when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // check on parent shards Thread.sleep(50L); @@ -657,7 +658,7 @@ public class ShardConsumerTest { } assertThat(consumer.getCurrentState(), equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE)); - assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.ZOMBIE))); + assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.TERMINATE))); verify(getRecordsCache).shutdown(); @@ -681,7 +682,7 @@ public class ShardConsumerTest { int numRecs = 10; BigInteger startSeqNum = BigInteger.ONE; String streamShardId = "kinesis-0-0"; - String testConcurrencyToken = "testToken"; + String testConcurrencyToken = UUID.randomUUID().toString(); List shardList = KinesisLocalFileDataCreator.createShardList(3, "kinesis-0-", startSeqNum); // Close the shard so that shutdown is called with reason terminate shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( @@ -749,7 +750,12 @@ public class ShardConsumerTest { shardSyncer, shardSyncStrategy); - when(shardSyncStrategy.onShardConsumerShutDown(shardList)).thenReturn(new TaskResult(null)); + List parentShardIds = new ArrayList<>(); + parentShardIds.add(shardInfo.getShardId()); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(createLease(shardInfo.getShardId(), + "leaseOwner", + parentShardIds)); + when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // check on parent shards @@ -939,7 +945,7 @@ public class ShardConsumerTest { @SuppressWarnings("unchecked") @Test public final void testConsumeShardInitializedWithPendingCheckpoint() throws Exception { - ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + ShardInfo shardInfo = new ShardInfo("s-0-0", UUID.randomUUID().toString(), null, ExtendedSequenceNumber.TRIM_HORIZON); StreamConfig streamConfig = new StreamConfig(streamProxy, 1, @@ -967,6 +973,7 @@ public class ShardConsumerTest { final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = new ExtendedSequenceNumber("999"); + when(streamProxy.getIterator(anyString(), anyString(), anyString())).thenReturn("startingIterator"); when(leaseManager.getLease(anyString())).thenReturn(null); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(config.getRecordsFetcherFactory()).thenReturn(new SimpleRecordsFetcherFactory()); @@ -1125,6 +1132,14 @@ public class ShardConsumerTest { return userRecords; } + private KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey(leaseKey); + lease.setLeaseOwner(leaseOwner); + lease.setParentShardIds(parentShardIds); + return lease; + } + Matcher initializationInputMatcher(final ExtendedSequenceNumber checkpoint, final ExtendedSequenceNumber pendingCheckpoint) { return new TypeSafeMatcher() { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 04fadd88..cbfdf54a 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -24,11 +24,17 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.UUID; import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; import com.amazonaws.services.kinesis.model.Shard; @@ -60,7 +66,7 @@ public class ShutdownTaskTest { InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); Set defaultParentShardIds = new HashSet<>(); - String defaultConcurrencyToken = "testToken4398"; + String defaultConcurrencyToken = UUID.randomUUID().toString(); String defaultShardId = "shardId-0"; ShardInfo defaultShardInfo = new ShardInfo(defaultShardId, defaultConcurrencyToken, @@ -70,10 +76,16 @@ public class ShutdownTaskTest { ShardSyncer shardSyncer = new KinesisShardSyncer(new KinesisLeaseCleanupValidator()); + @Mock + private IKinesisProxy kinesisProxy; @Mock private GetRecordsCache getRecordsCache; @Mock private ShardSyncStrategy shardSyncStrategy; + @Mock + private ILeaseManager leaseManager; + @Mock + private KinesisClientLibLeaseCoordinator leaseCoordinator; /** * @throws java.lang.Exception @@ -95,6 +107,10 @@ public class ShutdownTaskTest { @Before public void setUp() throws Exception { doNothing().when(getRecordsCache).shutdown(); + final KinesisClientLease parentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn(parentLease); + when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); } /** @@ -111,12 +127,6 @@ public class ShutdownTaskTest { public final void testCallWhenApplicationDoesNotCheckpoint() { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - List shards = constructShardListForGraphA(); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(true, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; @@ -132,31 +142,29 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + constructChildShards()); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof IllegalArgumentException); + final String expectedExceptionMessage = "Application didn't checkpoint at end of shard shardId-0. " + + "Application must checkpoint upon shutdown. See IRecordProcessor.shutdown javadocs for more information."; + Assert.assertEquals(expectedExceptionMessage, result.getException().getMessage()); } /** * Test method for {@link ShutdownTask#call()}. */ @Test - public final void testCallWhenSyncingShardsThrows() { + public final void testCallWhenCreatingLeaseThrows() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(true, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(new KinesisClientLibIOException(""))); + final String exceptionMessage = "InvalidStateException is thrown."; + when(leaseManager.createLeaseIfNotExists(any(KinesisClientLease.class))).thenThrow(new InvalidStateException(exceptionMessage)); ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -169,30 +177,21 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + constructChildShards()); TaskResult result = task.call(); - verify(shardSyncStrategy).onShardConsumerShutDown(shards); - Assert.assertNotNull(result.getException()); - Assert.assertTrue(result.getException() instanceof KinesisClientLibIOException); verify(getRecordsCache).shutdown(); + verify(leaseCoordinator).dropLease(any(KinesisClientLease.class)); + Assert.assertNull(result.getException()); } @Test - public final void testCallWhenShardEnd() { + public final void testCallWhenShardEnd() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(true, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(null)); ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -205,36 +204,27 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + constructChildShards()); TaskResult result = task.call(); - verify(shardSyncStrategy).onShardConsumerShutDown(shards); - verify(kinesisProxy, times(1)).verifyShardClosure(anyString()); + verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); + verify(leaseCoordinator).updateLease(any(KinesisClientLease.class), any(UUID.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); - verify(leaseCoordinator, never()).dropLease(any()); } @Test - public final void testCallWhenFalseShardEnd() { + public final void testCallWhenShardNotFound() throws Exception { ShardInfo shardInfo = new ShardInfo("shardId-4", defaultConcurrencyToken, defaultParentShardIds, ExtendedSequenceNumber.LATEST); RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(false, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(new KinesisClientLease()); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(null)); - ShutdownTask task = new ShutdownTask(shardInfo, defaultRecordProcessor, checkpointer, @@ -247,31 +237,23 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + Collections.emptyList()); TaskResult result = task.call(); - verify(shardSyncStrategy, never()).onShardConsumerShutDown(shards); - verify(kinesisProxy, times(1)).verifyShardClosure(anyString()); + verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); + verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); - verify(leaseCoordinator).dropLease(any()); } @Test - public final void testCallWhenLeaseLost() { + public final void testCallWhenLeaseLost() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); when(checkpointer.getLastCheckpointValue()).thenReturn(new ExtendedSequenceNumber("3298")); - List shards = constructShardListForGraphA(); - IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); - when(kinesisProxy.getShardList()).thenReturn(shards); - when(kinesisProxy.verifyShardClosure(anyString())).thenReturn(new ShardListWrappingShardClosureVerificationResponse(false, shards)); - KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); - when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); boolean cleanupLeasesOfCompletedShards = false; boolean ignoreUnexpectedChildShards = false; - when(shardSyncStrategy.onShardConsumerShutDown(shards)).thenReturn(new TaskResult(null)); ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -284,13 +266,13 @@ public class ShutdownTaskTest { TASK_BACKOFF_TIME_MILLIS, getRecordsCache, shardSyncer, - shardSyncStrategy); + shardSyncStrategy, + Collections.emptyList()); TaskResult result = task.call(); - verify(shardSyncStrategy, never()).onShardConsumerShutDown(shards); - verify(kinesisProxy, never()).getShardList(); + verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); + verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); - verify(leaseCoordinator, never()).dropLease(any()); } /** @@ -299,10 +281,39 @@ public class ShutdownTaskTest { @Test public final void testGetTaskType() { KinesisClientLibLeaseCoordinator leaseCoordinator = mock(KinesisClientLibLeaseCoordinator.class); - ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, leaseCoordinator, 0, getRecordsCache, shardSyncer, shardSyncStrategy); + ShutdownTask task = new ShutdownTask(null, null, null, null, + null, null, false, + false, leaseCoordinator, 0, + getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList()); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } + private List constructChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(defaultShardId); + + ChildShard leftChild = new ChildShard(); + leftChild.setShardId("ShardId-1"); + leftChild.setParentShards(parentShards); + leftChild.setHashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")); + childShards.add(leftChild); + + ChildShard rightChild = new ChildShard(); + rightChild.setShardId("ShardId-2"); + rightChild.setParentShards(parentShards); + rightChild.setHashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")); + childShards.add(rightChild); + return childShards; + } + + private KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + KinesisClientLease lease = new KinesisClientLease(); + lease.setLeaseKey(leaseKey); + lease.setLeaseOwner(leaseOwner); + lease.setParentShardIds(parentShardIds); + return lease; + } /* * Helper method to construct a shard list for graph A. Graph A is defined below. diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java index ed57eedf..9ade05bd 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java @@ -25,6 +25,7 @@ import java.nio.charset.Charset; import java.nio.charset.CharsetEncoder; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.HashSet; @@ -33,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.util.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -388,14 +390,33 @@ public class KinesisLocalFileProxy implements IKinesisProxy { */ response.setNextShardIterator(serializeIterator(iterator.shardId, lastRecordsSeqNo.add(BigInteger.ONE) .toString())); + response.setChildShards(Collections.emptyList()); LOG.debug("Returning a non null iterator for shard " + iterator.shardId); } else { + response.setChildShards(constructChildShards(iterator)); LOG.info("Returning null iterator for shard " + iterator.shardId); } return response; } + private List constructChildShards(IteratorInfo iterator) { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(iterator.shardId); + + ChildShard leftChild = new ChildShard(); + leftChild.setShardId("ShardId-1"); + leftChild.setParentShards(parentShards); + childShards.add(leftChild); + + ChildShard rightChild = new ChildShard(); + rightChild.setShardId("ShardId-2"); + rightChild.setParentShards(parentShards); + childShards.add(rightChild); + return childShards; + } + /** * {@inheritDoc} */ diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java index 4f4fdbca..d562a639 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java @@ -30,7 +30,7 @@ public class KinesisClientLeaseBuilder { private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); - private Set childShardIds = new HashSet<>(); + private Set childShardIds = new HashSet<>(); private HashKeyRangeForLease hashKeyRangeForLease; public KinesisClientLeaseBuilder withLeaseKey(String leaseKey) { @@ -90,7 +90,6 @@ public class KinesisClientLeaseBuilder { public KinesisClientLease build() { return new KinesisClientLease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, - hashKeyRangeForLease); + checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds, childShardIds, hashKeyRangeForLease); } } \ No newline at end of file From 36ae2f2428fa2133bba991f7b08a5c42c13ed559 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 8 Jul 2020 17:45:02 -0700 Subject: [PATCH 10/43] Bug fix: No longer need full shard sync for shardEnd --- .../services/kinesis/clientlibrary/lib/worker/Worker.java | 4 ---- .../kinesis/clientlibrary/lib/worker/WorkerTest.java | 6 +++++- .../clientlibrary/proxies/KinesisLocalFileProxy.java | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 9cdb71b5..f99d24d0 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -689,10 +689,6 @@ public class Worker implements Runnable { assignedShards.add(shardInfo); } - if (foundCompletedShard) { - shardSyncStrategy.onFoundCompletedShard(); - } - // clean up shard consumers for unassigned shards cleanupShardConsumers(assignedShards); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index b7e1dd51..a714f0fe 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -2184,7 +2184,8 @@ public class WorkerTest { private List createShardListWithOneSplit() { List shards = new ArrayList(); SequenceNumberRange range0 = ShardObjectHelper.newSequenceNumberRange("39428", "987324"); - SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("987325", null); + SequenceNumberRange range1 = ShardObjectHelper.newSequenceNumberRange("39428", "100000"); + SequenceNumberRange range2 = ShardObjectHelper.newSequenceNumberRange("100001", "987324"); HashKeyRange keyRange = ShardObjectHelper.newHashKeyRange(ShardObjectHelper.MIN_HASH_KEY, ShardObjectHelper.MAX_HASH_KEY); Shard shard0 = ShardObjectHelper.newShard("shardId-0", null, null, range0, keyRange); @@ -2193,6 +2194,9 @@ public class WorkerTest { Shard shard1 = ShardObjectHelper.newShard("shardId-1", "shardId-0", null, range1, keyRange); shards.add(shard1); + Shard shard2 = ShardObjectHelper.newShard("shardId-2", null, "shardId-0", range2, keyRange); + shards.add(shard2); + return shards; } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java index 9ade05bd..fe922520 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisLocalFileProxy.java @@ -406,12 +406,12 @@ public class KinesisLocalFileProxy implements IKinesisProxy { parentShards.add(iterator.shardId); ChildShard leftChild = new ChildShard(); - leftChild.setShardId("ShardId-1"); + leftChild.setShardId("shardId-1"); leftChild.setParentShards(parentShards); childShards.add(leftChild); ChildShard rightChild = new ChildShard(); - rightChild.setShardId("ShardId-2"); + rightChild.setShardId("shardId-2"); rightChild.setParentShards(parentShards); childShards.add(rightChild); return childShards; From 058403f943f108112f91dffee8140f95a12b2c3f Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 9 Jul 2020 11:20:37 -0700 Subject: [PATCH 11/43] addressing comments --- .../services/kinesis/clientlibrary/lib/worker/WorkerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index a714f0fe..a7ca1151 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -2194,7 +2194,7 @@ public class WorkerTest { Shard shard1 = ShardObjectHelper.newShard("shardId-1", "shardId-0", null, range1, keyRange); shards.add(shard1); - Shard shard2 = ShardObjectHelper.newShard("shardId-2", null, "shardId-0", range2, keyRange); + Shard shard2 = ShardObjectHelper.newShard("shardId-2", "shardId-0", null, range2, keyRange); shards.add(shard2); return shards; From 995511afe5c68a1ed8f0329c98db61e90868a763 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 9 Jul 2020 16:03:49 -0700 Subject: [PATCH 12/43] Fixing updateParentShardupdateCurrentLeaseWithChildShards method --- .../lib/worker/PeriodicShardSyncManager.java | 5 +++-- .../kinesis/clientlibrary/lib/worker/ShutdownTask.java | 6 ++---- .../clientlibrary/lib/worker/ShutdownTaskTest.java | 8 ++++---- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java index 32fdec54..c532a10d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java @@ -26,6 +26,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import com.amazonaws.services.cloudwatch.model.StandardUnit; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; @@ -180,8 +181,8 @@ class PeriodicShardSyncManager { try { final ShardSyncResponse shardSyncResponse = checkForShardSync(); - MetricsHelper.addSuccessAndLatency(runStartMillis, shardSyncResponse.shouldDoShardSync(), MetricsLevel.SUMMARY); - MetricsHelper.addSuccessAndLatency(runStartMillis, shardSyncResponse.isHoleDetected(), MetricsLevel.SUMMARY); + MetricsHelper.getMetricsScope().addData("ShouldDoShardSync", shardSyncResponse.shouldDoShardSync() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY); + MetricsHelper.getMetricsScope().addData("HashRangeHoleDetected", shardSyncResponse.isHoleDetected() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY); if (shardSyncResponse.shouldDoShardSync()) { LOG.info("Periodic shard syncer initiating shard sync due to the reason - " + shardSyncResponse.reasonForDecision()); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index 71cf3b9d..8c58f654 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; @@ -210,10 +211,7 @@ class ShutdownTask implements ITask { final Set childShardIds = childShards.stream().map(ChildShard::getShardId).collect(Collectors.toSet()); currentLease.setChildShardIds(childShardIds); - final boolean updateResult = leaseCoordinator.updateLease(currentLease, UUID.fromString(shardInfo.getConcurrencyToken())); - if (!updateResult) { - throw new InvalidStateException("Failed to update parent lease with child shard information for shard " + shardInfo.getShardId()); - } + leaseCoordinator.getLeaseManager().updateLeaseWithMetaInfo(currentLease, UpdateField.CHILD_SHARDS); LOG.info("Shard " + shardInfo.getShardId() + ": Updated current lease with child shard information: " + currentLease.getLeaseKey()); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index cbfdf54a..7a780eff 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -34,6 +34,7 @@ import java.util.UUID; import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; @@ -110,7 +111,6 @@ public class ShutdownTaskTest { final KinesisClientLease parentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn(parentLease); - when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); } /** @@ -208,7 +208,7 @@ public class ShutdownTaskTest { constructChildShards()); TaskResult result = task.call(); verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); - verify(leaseCoordinator).updateLease(any(KinesisClientLease.class), any(UUID.class)); + verify(leaseManager).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); } @@ -241,7 +241,7 @@ public class ShutdownTaskTest { Collections.emptyList()); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); - verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); + verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); } @@ -270,7 +270,7 @@ public class ShutdownTaskTest { Collections.emptyList()); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); - verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); + verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); } From a1eafbb69a10a01886c822cac12d5e3ff50c4346 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Fri, 10 Jul 2020 10:43:29 -0700 Subject: [PATCH 13/43] Addressing comments --- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index d5749133..bd5fac5d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -181,7 +181,7 @@ public class KinesisClientLibConfiguration { public static final long LEASES_RECOVERY_AUDITOR_EXECUTION_FREQUENCY_MILLIS = 2 * 60 * 1000L; /** - * Default Lease Recovery Auditor inconsistency confidence threshold for running full shard sync for HARD_END ShardSyncStrategyType. + * Default Lease Recovery Auditor inconsistency confidence threshold for running full shard sync for SHARD_END ShardSyncStrategyType. */ public static final int LEASES_RECOVERY_AUDITOR_INCONSISTENCY_CONFIDENCE_THRESHOLD = 3; From 5bf24bda431a1b902210b00f3a74de232138214e Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Wed, 15 Jul 2020 12:33:53 -0700 Subject: [PATCH 14/43] fix for premature childShard lease creation --- .../lib/worker/ShutdownTask.java | 114 ++++++++------- .../lib/worker/ShutdownTaskTest.java | 131 ++++++++++++++++-- 2 files changed, 188 insertions(+), 57 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index 71cf3b9d..ed9de7e2 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -14,9 +14,11 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.BlockedOnParentShardException; import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; @@ -32,8 +34,8 @@ import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.google.common.annotations.VisibleForTesting; import java.util.List; +import java.util.Objects; import java.util.Set; -import java.util.UUID; import java.util.stream.Collectors; /** @@ -44,6 +46,7 @@ class ShutdownTask implements ITask { private static final Log LOG = LogFactory.getLog(ShutdownTask.class); private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown"; + private int retryLeftForValidParentState = 10; private final ShardInfo shardInfo; private final IRecordProcessor recordProcessor; @@ -107,43 +110,8 @@ class ShutdownTask implements ITask { try { LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: " + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); - ShutdownReason localReason = reason; - /* - * Revalidate if the current shard is closed before shutting down the shard consumer with reason SHARD_END - * If current shard is not closed, shut down the shard consumer with reason LEASE_LOST that allows active - * workers to contend for the lease of this shard. - */ - if(localReason == ShutdownReason.TERMINATE) { - // Create new lease for the child shards if they don't exist. - // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. - // This would happen when KinesisDataFetcher catches ResourceNotFound exception. - // In this case, KinesisDataFetcher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. - // This scenario could happen when customer deletes the stream while leaving the KCL application running. - try { - if (!CollectionUtils.isNullOrEmpty(childShards)) { - createLeasesForChildShardsIfNotExist(); - updateCurrentLeaseWithChildShards(); - } else { - LOG.warn("Shard " + shardInfo.getShardId() - + ": Shutting down consumer with SHARD_END reason without creating leases for child shards."); - } - } catch (InvalidStateException e) { - // If invalidStateException happens, it indicates we are missing childShard related information. - // In this scenario, we should shutdown the shardConsumer with ZOMBIE reason to allow other worker to take the lease and retry getting - // childShard information in the processTask. - localReason = ShutdownReason.ZOMBIE; - dropLease(); - LOG.warn("Shard " + shardInfo.getShardId() + ": Exception happened while shutting down shardConsumer with TERMINATE reason. " + - "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); - } - } - // If we reached end of the shard, set sequence number to SHARD_END. - if (localReason == ShutdownReason.TERMINATE) { - recordProcessorCheckpointer.setSequenceNumberAtShardEnd( - recordProcessorCheckpointer.getLargestPermittedCheckpointValue()); - recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); - } + ShutdownReason localReason = attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(reason); final ShutdownInput shutdownInput = new ShutdownInput() .withShutdownReason(localReason) @@ -191,7 +159,67 @@ class ShutdownTask implements ITask { return new TaskResult(exception); } + private ShutdownReason attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(ShutdownReason originalReason) + throws DependencyException, ProvisionedThroughputException { + ShutdownReason shutdownReason = originalReason; + if(originalReason == ShutdownReason.TERMINATE) { + // For TERMINATE shutdown reason, try to create and persist childShard leases before setting checkpoint. + try { + final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + if (currentLease == null) { + throw new InvalidStateException(shardInfo.getShardId() + + " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner."); + } + if (!CollectionUtils.isNullOrEmpty(childShards)) { + createLeasesForChildShardsIfNotExist(); + updateCurrentLeaseWithChildShards(currentLease); + recordProcessorCheckpointer.setSequenceNumberAtShardEnd( + recordProcessorCheckpointer.getLargestPermittedCheckpointValue()); + recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); + } else { + LOG.warn("Shard " + shardInfo.getShardId() + + ": Shutting down consumer with SHARD_END reason without creating leases for child shards."); + } + } catch (InvalidStateException e) { + // If invalidStateException happens, it indicates we are missing childShard related information. + // In this scenario, we should shutdown the shardConsumer with ZOMBIE reason to allow other worker to take the lease and retry getting + // childShard information in the processTask. + shutdownReason = ShutdownReason.ZOMBIE; + dropLease(); + LOG.warn("Shard " + shardInfo.getShardId() + ": Exception happened while shutting down shardConsumer with TERMINATE reason. " + + "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); + } + + } + return shutdownReason; + } + private void createLeasesForChildShardsIfNotExist() throws InvalidStateException, DependencyException, ProvisionedThroughputException { + // For child shard resulted from merge of two parent shards, verify if both the parents are either present or + // not present in the lease table before creating the lease entry. + if (!CollectionUtils.isNullOrEmpty(childShards) && childShards.size() == 1) { + final ChildShard childShard = childShards.get(0); + final List parentLeaseKeys = childShard.getParentShards(); + + if (parentLeaseKeys.size() != 2) { + throw new InvalidStateException("Shard " + shardInfo.getShardId()+ "'s only child shard " + childShard + + " does not contain other parent information."); + } else { + boolean isValidLeaseTableState = Objects.isNull(leaseCoordinator.getLeaseManager().getLease(parentLeaseKeys.get(0))) == + Objects.isNull(leaseCoordinator.getLeaseManager().getLease(parentLeaseKeys.get(1))); + if (!isValidLeaseTableState) { + if(--retryLeftForValidParentState >= 0) { + throw new BlockedOnParentShardException( + "Shard " + shardInfo.getShardId() + "'s only child shard " + childShard + + " has partial parent information in lease table. Hence deferring lease creation of child shard."); + } else { + throw new InvalidStateException("Shard " + shardInfo.getShardId() + "'s only child shard " + childShard + + " has partial parent information in lease table."); + } + } + } + } + // Attempt create leases for child shards. for (ChildShard childShard : childShards) { final String leaseKey = childShard.getShardId(); if (leaseCoordinator.getLeaseManager().getLease(leaseKey) == null) { @@ -202,18 +230,10 @@ class ShutdownTask implements ITask { } } - private void updateCurrentLeaseWithChildShards() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); - if (currentLease == null) { - throw new InvalidStateException("Failed to retrieve current lease for shard " + shardInfo.getShardId()); - } + private void updateCurrentLeaseWithChildShards(KinesisClientLease currentLease) throws DependencyException, InvalidStateException, ProvisionedThroughputException { final Set childShardIds = childShards.stream().map(ChildShard::getShardId).collect(Collectors.toSet()); - currentLease.setChildShardIds(childShardIds); - final boolean updateResult = leaseCoordinator.updateLease(currentLease, UUID.fromString(shardInfo.getConcurrencyToken())); - if (!updateResult) { - throw new InvalidStateException("Failed to update parent lease with child shard information for shard " + shardInfo.getShardId()); - } + leaseCoordinator.getLeaseManager().updateLeaseWithMetaInfo(currentLease, UpdateField.CHILD_SHARDS); LOG.info("Shard " + shardInfo.getShardId() + ": Updated current lease with child shard information: " + currentLease.getLeaseKey()); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index cbfdf54a..6400b839 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -14,6 +14,9 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doNothing; @@ -31,9 +34,11 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.BlockedOnParentShardException; import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.impl.UpdateField; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; @@ -72,7 +77,6 @@ public class ShutdownTaskTest { defaultConcurrencyToken, defaultParentShardIds, ExtendedSequenceNumber.LATEST); - IRecordProcessor defaultRecordProcessor = new TestStreamlet(); ShardSyncer shardSyncer = new KinesisShardSyncer(new KinesisLeaseCleanupValidator()); @@ -86,6 +90,8 @@ public class ShutdownTaskTest { private ILeaseManager leaseManager; @Mock private KinesisClientLibLeaseCoordinator leaseCoordinator; + @Mock + private IRecordProcessor defaultRecordProcessor; /** * @throws java.lang.Exception @@ -110,7 +116,6 @@ public class ShutdownTaskTest { final KinesisClientLease parentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn(parentLease); - when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); } /** @@ -143,9 +148,9 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - constructChildShards()); + constructSplitChildShards()); TaskResult result = task.call(); - Assert.assertNotNull(result.getException()); + assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof IllegalArgumentException); final String expectedExceptionMessage = "Application didn't checkpoint at end of shard shardId-0. " + "Application must checkpoint upon shutdown. See IRecordProcessor.shutdown javadocs for more information."; @@ -178,13 +183,104 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - constructChildShards()); + constructSplitChildShards()); TaskResult result = task.call(); verify(getRecordsCache).shutdown(); verify(leaseCoordinator).dropLease(any(KinesisClientLease.class)); Assert.assertNull(result.getException()); } + @Test + public final void testCallWhenParentInfoNotPresentInLease() throws Exception { + RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); + when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + boolean cleanupLeasesOfCompletedShards = false; + boolean ignoreUnexpectedChildShards = false; + + KinesisClientLease currentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); + KinesisClientLease adjacentParentLease = createLease("ShardId-1", "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn( currentLease); + when(leaseManager.getLease(defaultShardId)).thenReturn(currentLease); + when(leaseManager.getLease("ShardId-1")).thenReturn(null, null, null, null, null, adjacentParentLease); + + ShutdownTask task = new ShutdownTask(defaultShardInfo, + defaultRecordProcessor, + checkpointer, + ShutdownReason.TERMINATE, + kinesisProxy, + INITIAL_POSITION_TRIM_HORIZON, + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, + leaseCoordinator, + TASK_BACKOFF_TIME_MILLIS, + getRecordsCache, + shardSyncer, + shardSyncStrategy, + constructMergeChildShards()); + + // 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(getRecordsCache, never()).shutdown(); + verify(defaultRecordProcessor, never()).shutdown(any(ShutdownInput.class)); + } + + // Make next attempt with complete parent info in lease table + TaskResult result = task.call(); + assertNull(result.getException()); + verify(getRecordsCache).shutdown(); + verify(defaultRecordProcessor).shutdown(any(ShutdownInput.class)); + verify(leaseCoordinator, never()).dropLease(currentLease); + } + + @Test + public final void testCallTriggersLeaseLossWhenParentInfoNotPresentInLease() throws Exception { + RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); + when(checkpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); + boolean cleanupLeasesOfCompletedShards = false; + boolean ignoreUnexpectedChildShards = false; + + KinesisClientLease currentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); + when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn( currentLease); + when(leaseManager.getLease(defaultShardId)).thenReturn(currentLease); + when(leaseManager.getLease("ShardId-1")).thenReturn(null, null, null, null, null, null, null, null, null, null, null); + + ShutdownTask task = new ShutdownTask(defaultShardInfo, + defaultRecordProcessor, + checkpointer, + ShutdownReason.TERMINATE, + kinesisProxy, + INITIAL_POSITION_TRIM_HORIZON, + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, + leaseCoordinator, + TASK_BACKOFF_TIME_MILLIS, + getRecordsCache, + shardSyncer, + shardSyncStrategy, + constructMergeChildShards()); + + 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(getRecordsCache, never()).shutdown(); + verify(defaultRecordProcessor, never()).shutdown(any(ShutdownInput.class)); + } + + TaskResult result = task.call(); + assertNull(result.getException()); + verify(getRecordsCache).shutdown(); + verify(defaultRecordProcessor).shutdown(any(ShutdownInput.class)); + verify(leaseCoordinator).dropLease(currentLease); + } + @Test public final void testCallWhenShardEnd() throws Exception { RecordProcessorCheckpointer checkpointer = mock(RecordProcessorCheckpointer.class); @@ -205,10 +301,10 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - constructChildShards()); + constructSplitChildShards()); TaskResult result = task.call(); verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); - verify(leaseCoordinator).updateLease(any(KinesisClientLease.class), any(UUID.class)); + verify(leaseManager).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); } @@ -241,7 +337,7 @@ public class ShutdownTaskTest { Collections.emptyList()); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); - verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); + verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); } @@ -270,7 +366,7 @@ public class ShutdownTaskTest { Collections.emptyList()); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); - verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); + verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); Assert.assertNull(result.getException()); verify(getRecordsCache).shutdown(); } @@ -288,7 +384,7 @@ public class ShutdownTaskTest { Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } - private List constructChildShards() { + private List constructSplitChildShards() { List childShards = new ArrayList<>(); List parentShards = new ArrayList<>(); parentShards.add(defaultShardId); @@ -307,6 +403,21 @@ public class ShutdownTaskTest { return childShards; } + private List constructMergeChildShards() { + List childShards = new ArrayList<>(); + List parentShards = new ArrayList<>(); + parentShards.add(defaultShardId); + parentShards.add("ShardId-1"); + + ChildShard childShard = new ChildShard(); + childShard.setShardId("ShardId-2"); + childShard.setParentShards(parentShards); + childShard.setHashKeyRange(ShardObjectHelper.newHashKeyRange("0", "99")); + childShards.add(childShard); + + return childShards; + } + private KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { KinesisClientLease lease = new KinesisClientLease(); lease.setLeaseKey(leaseKey); From 089c6ab18ef9429f5e808cf4db722a999399c2a6 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Thu, 16 Jul 2020 16:27:25 -0400 Subject: [PATCH 15/43] Making isLeaseTableEmpty use one paginated scan call. (#75) Co-authored-by: Joshua Kim --- .../kinesis/leases/impl/LeaseManager.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index 7fe4551a..e5860870 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -240,7 +240,7 @@ public class LeaseManager implements ILeaseManager { */ @Override public boolean isLeaseTableEmpty() throws DependencyException, InvalidStateException, ProvisionedThroughputException { - return list(1).isEmpty(); + return list(1, 1).isEmpty(); } /** @@ -253,6 +253,20 @@ public class LeaseManager implements ILeaseManager { * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity */ List list(Integer limit) throws DependencyException, InvalidStateException, ProvisionedThroughputException { + return list(limit, Integer.MAX_VALUE); + } + + /** + * List with the given page size, up to a limit of paginated calls. + * + * @param limit number of items to consider at a time - used by integration tests to force paging. + * @param maxPages max number of paginated scan calls. + * @return list of leases + * @throws InvalidStateException if table does not exist + * @throws DependencyException if DynamoDB scan fail in an unexpected way + * @throws ProvisionedThroughputException if DynamoDB scan fail due to exceeded capacity + */ + private List list(Integer limit, Integer maxPages) throws InvalidStateException, ProvisionedThroughputException, DependencyException { if (LOG.isDebugEnabled()) { LOG.debug("Listing leases from table " + table); } @@ -277,7 +291,7 @@ public class LeaseManager implements ILeaseManager { } Map lastEvaluatedKey = scanResult.getLastEvaluatedKey(); - if (lastEvaluatedKey == null) { + if (lastEvaluatedKey == null || --maxPages <= 0) { // Signify that we're done. scanResult = null; if (LOG.isDebugEnabled()) { From 8a296a5aa3c50a012f3a1ad27bea1f4bf4ffe65e Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 21 Jul 2020 11:08:56 -0700 Subject: [PATCH 16/43] change the retry logic --- .../lib/worker/ShutdownTask.java | 15 ++- .../lib/worker/ShutdownTaskTest.java | 96 +++++++++++++------ 2 files changed, 79 insertions(+), 32 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index ed9de7e2..9af4645f 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -35,6 +35,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.List; import java.util.Objects; +import java.util.Random; import java.util.Set; import java.util.stream.Collectors; @@ -46,7 +47,8 @@ class ShutdownTask implements ITask { private static final Log LOG = LogFactory.getLog(ShutdownTask.class); private static final String RECORD_PROCESSOR_SHUTDOWN_METRIC = "RecordProcessor.shutdown"; - private int retryLeftForValidParentState = 10; + @VisibleForTesting + static final int RETRY_RANDOM_MAX_RANGE = 10; private final ShardInfo shardInfo; private final IRecordProcessor recordProcessor; @@ -208,7 +210,7 @@ class ShutdownTask implements ITask { boolean isValidLeaseTableState = Objects.isNull(leaseCoordinator.getLeaseManager().getLease(parentLeaseKeys.get(0))) == Objects.isNull(leaseCoordinator.getLeaseManager().getLease(parentLeaseKeys.get(1))); if (!isValidLeaseTableState) { - if(--retryLeftForValidParentState >= 0) { + if(!isOneInNProbability(RETRY_RANDOM_MAX_RANGE)) { throw new BlockedOnParentShardException( "Shard " + shardInfo.getShardId() + "'s only child shard " + childShard + " has partial parent information in lease table. Hence deferring lease creation of child shard."); @@ -230,6 +232,15 @@ class ShutdownTask implements ITask { } } + /** + * Returns true for 1 in N probability. + */ + @VisibleForTesting + boolean isOneInNProbability(int n) { + Random r = new Random(); + return 1 == r.nextInt((n - 1) + 1) + 1; + } + private void updateCurrentLeaseWithChildShards(KinesisClientLease currentLease) throws DependencyException, InvalidStateException, ProvisionedThroughputException { final Set childShardIds = childShards.stream().map(ChildShard::getShardId).collect(Collectors.toSet()); currentLease.setChildShardIds(childShardIds); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 6400b839..d9226f44 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -22,6 +22,7 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -60,6 +61,7 @@ import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownTask.RETRY_RANDOM_MAX_RANGE; /** * @@ -204,34 +206,51 @@ public class ShutdownTaskTest { when(leaseManager.getLease(defaultShardId)).thenReturn(currentLease); when(leaseManager.getLease("ShardId-1")).thenReturn(null, null, null, null, null, adjacentParentLease); - ShutdownTask task = new ShutdownTask(defaultShardInfo, - defaultRecordProcessor, - checkpointer, - ShutdownReason.TERMINATE, - kinesisProxy, - INITIAL_POSITION_TRIM_HORIZON, - cleanupLeasesOfCompletedShards, - ignoreUnexpectedChildShards, - leaseCoordinator, - TASK_BACKOFF_TIME_MILLIS, - getRecordsCache, - shardSyncer, - shardSyncStrategy, - constructMergeChildShards()); - // Make first 5 attempts with partial parent info in lease table for (int i = 0; i < 5; i++) { + ShutdownTask task = spy(new ShutdownTask(defaultShardInfo, + defaultRecordProcessor, + checkpointer, + ShutdownReason.TERMINATE, + kinesisProxy, + INITIAL_POSITION_TRIM_HORIZON, + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, + leaseCoordinator, + TASK_BACKOFF_TIME_MILLIS, + getRecordsCache, + shardSyncer, + shardSyncStrategy, + constructMergeChildShards())); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); assertTrue(result.getException() instanceof BlockedOnParentShardException); assertTrue(result.getException().getMessage().contains("has partial parent information in lease table")); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); verify(getRecordsCache, never()).shutdown(); verify(defaultRecordProcessor, never()).shutdown(any(ShutdownInput.class)); } // Make next attempt with complete parent info in lease table + ShutdownTask task = spy(new ShutdownTask(defaultShardInfo, + defaultRecordProcessor, + checkpointer, + ShutdownReason.TERMINATE, + kinesisProxy, + INITIAL_POSITION_TRIM_HORIZON, + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, + leaseCoordinator, + TASK_BACKOFF_TIME_MILLIS, + getRecordsCache, + shardSyncer, + shardSyncStrategy, + constructMergeChildShards())); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNull(result.getException()); + verify(task, never()).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); verify(getRecordsCache).shutdown(); verify(defaultRecordProcessor).shutdown(any(ShutdownInput.class)); verify(leaseCoordinator, never()).dropLease(currentLease); @@ -250,32 +269,49 @@ public class ShutdownTaskTest { when(leaseManager.getLease(defaultShardId)).thenReturn(currentLease); when(leaseManager.getLease("ShardId-1")).thenReturn(null, null, null, null, null, null, null, null, null, null, null); - ShutdownTask task = new ShutdownTask(defaultShardInfo, - defaultRecordProcessor, - checkpointer, - ShutdownReason.TERMINATE, - kinesisProxy, - INITIAL_POSITION_TRIM_HORIZON, - cleanupLeasesOfCompletedShards, - ignoreUnexpectedChildShards, - leaseCoordinator, - TASK_BACKOFF_TIME_MILLIS, - getRecordsCache, - shardSyncer, - shardSyncStrategy, - constructMergeChildShards()); - for (int i = 0; i < 10; i++) { + ShutdownTask task = spy(new ShutdownTask(defaultShardInfo, + defaultRecordProcessor, + checkpointer, + ShutdownReason.TERMINATE, + kinesisProxy, + INITIAL_POSITION_TRIM_HORIZON, + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, + leaseCoordinator, + TASK_BACKOFF_TIME_MILLIS, + getRecordsCache, + shardSyncer, + shardSyncStrategy, + constructMergeChildShards())); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); assertTrue(result.getException() instanceof BlockedOnParentShardException); assertTrue(result.getException().getMessage().contains("has partial parent information in lease table")); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); verify(getRecordsCache, never()).shutdown(); verify(defaultRecordProcessor, never()).shutdown(any(ShutdownInput.class)); } + ShutdownTask task = spy(new ShutdownTask(defaultShardInfo, + defaultRecordProcessor, + checkpointer, + ShutdownReason.TERMINATE, + kinesisProxy, + INITIAL_POSITION_TRIM_HORIZON, + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, + leaseCoordinator, + TASK_BACKOFF_TIME_MILLIS, + getRecordsCache, + shardSyncer, + shardSyncStrategy, + constructMergeChildShards())); + when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(true); TaskResult result = task.call(); assertNull(result.getException()); + verify(task, times(1)).isOneInNProbability(RETRY_RANDOM_MAX_RANGE); verify(getRecordsCache).shutdown(); verify(defaultRecordProcessor).shutdown(any(ShutdownInput.class)); verify(leaseCoordinator).dropLease(currentLease); From 6f16b168a4fbd559a4c76956421bfbd9e4c99293 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 27 Jul 2020 16:08:52 -0400 Subject: [PATCH 17/43] Lease cleanup v1.x (#73) * Moving lease cleanup to ShutdownTask. * Introducing LeaseCleanupManager with relevant configs. --- .../lib/worker/ConsumerStates.java | 5 +- .../worker/KinesisClientLibConfiguration.java | 230 ++++++++--- .../lib/worker/KinesisDataFetcher.java | 2 - .../worker/KinesisLeaseCleanupValidator.java | 1 + .../lib/worker/KinesisShardSyncer.java | 181 +-------- .../lib/worker/LeaseCleanupValidator.java | 1 + .../lib/worker/ShardConsumer.java | 60 +++ .../clientlibrary/lib/worker/ShardInfo.java | 2 +- .../lib/worker/ShutdownTask.java | 27 +- .../clientlibrary/lib/worker/Worker.java | 29 +- .../kinesis/leases/LeasePendingDeletion.java | 31 ++ .../leases/impl/LeaseCleanupManager.java | 369 ++++++++++++++++++ .../lib/worker/ConsumerStatesTest.java | 2 +- .../lib/worker/ShardConsumerTest.java | 62 +-- .../lib/worker/ShardObjectHelper.java | 18 +- .../lib/worker/ShardSyncerTest.java | 103 +---- .../lib/worker/ShutdownTaskTest.java | 20 +- .../leases/impl/LeaseCleanupManagerTest.java | 289 ++++++++++++++ .../kinesis/leases/impl/LeaseHelper.java | 44 +++ 19 files changed, 1104 insertions(+), 372 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/leases/LeasePendingDeletion.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseHelper.java diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index fc3400e8..5cf55dbf 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -199,7 +199,7 @@ class ConsumerStates { @Override public ConsumerState shutdownTransition(ShutdownReason shutdownReason) { - return ShardConsumerState.SHUTDOWN_COMPLETE.getConsumerState(); + return ShardConsumerState.SHUTTING_DOWN.getConsumerState(); } @Override @@ -531,7 +531,8 @@ class ConsumerStates { consumer.getLeaseCoordinator(), consumer.getTaskBackoffTimeMillis(), consumer.getGetRecordsCache(), consumer.getShardSyncer(), - consumer.getShardSyncStrategy(), consumer.getChildShards()); + consumer.getShardSyncStrategy(), consumer.getChildShards(), + consumer.getLeaseCleanupManager()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 86e7a496..52e73b3e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -14,6 +14,7 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import java.time.Duration; import java.util.Date; import java.util.Optional; import java.util.Set; @@ -89,6 +90,23 @@ public class KinesisClientLibConfiguration { */ public static final boolean DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION = true; + /** + * Interval to run lease cleanup thread in {@link LeaseCleanupManager}. + */ + private static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofHours(1).toMillis(); + + /** + * Threshold in millis at which to check if there are any completed leases (leases for shards which have been + * closed as a result of a resharding operation) that need to be cleaned up. + */ + private static final long DEFAULT_COMPLETED_LEASE_CLEANUP_THRESHOLD_MILLIS = Duration.ofMinutes(5).toMillis(); + + /** + * Threshold in millis at which to check if there are any garbage leases (leases for shards which no longer exist + * in the stream) that need to be cleaned up. + */ + private static final long DEFAULT_GARBAGE_LEASE_CLEANUP_THRESHOLD_MILLIS = Duration.ofMinutes(30).toMillis(); + /** * Backoff time in milliseconds for Amazon Kinesis Client Library tasks (in the event of failures). */ @@ -246,6 +264,9 @@ public class KinesisClientLibConfiguration { private ShardPrioritization shardPrioritization; private long shutdownGraceMillis; private ShardSyncStrategyType shardSyncStrategyType; + private long leaseCleanupIntervalMillis; + private long completedLeaseCleanupThresholdMillis; + private long garbageLeaseCleanupThresholdMillis; @Getter private Optional timeoutInSeconds = Optional.empty(); @@ -284,6 +305,7 @@ public class KinesisClientLibConfiguration { * @param credentialsProvider Provides credentials used to sign AWS requests * @param workerId Used to distinguish different workers/processes of a Kinesis application */ + @Deprecated public KinesisClientLibConfiguration(String applicationName, String streamName, AWSCredentialsProvider credentialsProvider, @@ -303,6 +325,7 @@ public class KinesisClientLibConfiguration { * @param cloudWatchCredentialsProvider Provides credentials used to access CloudWatch * @param workerId Used to distinguish different workers/processes of a Kinesis application */ + @Deprecated public KinesisClientLibConfiguration(String applicationName, String streamName, AWSCredentialsProvider kinesisCredentialsProvider, @@ -373,6 +396,7 @@ public class KinesisClientLibConfiguration { */ // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES + @Deprecated public KinesisClientLibConfiguration(String applicationName, String streamName, String kinesisEndpoint, @@ -444,6 +468,7 @@ public class KinesisClientLibConfiguration { */ // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES + @Deprecated public KinesisClientLibConfiguration(String applicationName, String streamName, String kinesisEndpoint, @@ -470,54 +495,14 @@ public class KinesisClientLibConfiguration { String regionName, long shutdownGraceMillis, BillingMode billingMode) { - // Check following values are greater than zero - checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis); - checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis); - checkIsValuePositive("ParentShardPollIntervalMillis", parentShardPollIntervalMillis); - checkIsValuePositive("ShardSyncIntervalMillis", shardSyncIntervalMillis); - checkIsValuePositive("MaxRecords", (long) maxRecords); - checkIsValuePositive("TaskBackoffTimeMillis", taskBackoffTimeMillis); - checkIsValuePositive("MetricsBufferTimeMills", metricsBufferTimeMillis); - checkIsValuePositive("MetricsMaxQueueSize", (long) metricsMaxQueueSize); - checkIsValuePositive("ShutdownGraceMillis", shutdownGraceMillis); - this.applicationName = applicationName; - this.tableName = applicationName; - this.streamName = streamName; - this.kinesisEndpoint = kinesisEndpoint; - this.dynamoDBEndpoint = dynamoDBEndpoint; - this.initialPositionInStream = initialPositionInStream; - this.kinesisCredentialsProvider = kinesisCredentialsProvider; - this.dynamoDBCredentialsProvider = dynamoDBCredentialsProvider; - this.cloudWatchCredentialsProvider = cloudWatchCredentialsProvider; - this.failoverTimeMillis = failoverTimeMillis; - this.maxRecords = maxRecords; - this.idleTimeBetweenReadsInMillis = idleTimeBetweenReadsInMillis; - this.callProcessRecordsEvenForEmptyRecordList = callProcessRecordsEvenForEmptyRecordList; - this.parentShardPollIntervalMillis = parentShardPollIntervalMillis; - this.shardSyncIntervalMillis = shardSyncIntervalMillis; - this.cleanupLeasesUponShardCompletion = cleanupTerminatedShardsBeforeExpiry; - this.workerIdentifier = workerId; - this.kinesisClientConfig = checkAndAppendKinesisClientLibUserAgent(kinesisClientConfig); - this.dynamoDBClientConfig = checkAndAppendKinesisClientLibUserAgent(dynamoDBClientConfig); - this.cloudWatchClientConfig = checkAndAppendKinesisClientLibUserAgent(cloudWatchClientConfig); - this.taskBackoffTimeMillis = taskBackoffTimeMillis; - this.metricsBufferTimeMillis = metricsBufferTimeMillis; - this.metricsMaxQueueSize = metricsMaxQueueSize; - this.metricsLevel = DEFAULT_METRICS_LEVEL; - this.metricsEnabledDimensions = DEFAULT_METRICS_ENABLED_DIMENSIONS; - this.validateSequenceNumberBeforeCheckpointing = validateSequenceNumberBeforeCheckpointing; - this.regionName = regionName; - this.maxLeasesForWorker = DEFAULT_MAX_LEASES_FOR_WORKER; - this.maxLeasesToStealAtOneTime = DEFAULT_MAX_LEASES_TO_STEAL_AT_ONE_TIME; - this.initialLeaseTableReadCapacity = DEFAULT_INITIAL_LEASE_TABLE_READ_CAPACITY; - this.initialLeaseTableWriteCapacity = DEFAULT_INITIAL_LEASE_TABLE_WRITE_CAPACITY; - this.initialPositionInStreamExtended = - InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); - this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; - this.shardSyncStrategyType = DEFAULT_SHARD_SYNC_STRATEGY_TYPE; - this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; - this.recordsFetcherFactory = new SimpleRecordsFetcherFactory(); - this.billingMode = billingMode; + + this(applicationName, streamName, kinesisEndpoint, dynamoDBEndpoint, initialPositionInStream, kinesisCredentialsProvider, + dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId, maxRecords, idleTimeBetweenReadsInMillis, + callProcessRecordsEvenForEmptyRecordList, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry, + kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig, taskBackoffTimeMillis, metricsBufferTimeMillis, + metricsMaxQueueSize, validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis, billingMode, + new SimpleRecordsFetcherFactory(), DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS, DEFAULT_COMPLETED_LEASE_CLEANUP_THRESHOLD_MILLIS, + DEFAULT_GARBAGE_LEASE_CLEANUP_THRESHOLD_MILLIS); } /** @@ -556,6 +541,7 @@ public class KinesisClientLibConfiguration { */ // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES + @Deprecated public KinesisClientLibConfiguration(String applicationName, String streamName, String kinesisEndpoint, @@ -581,6 +567,91 @@ public class KinesisClientLibConfiguration { boolean validateSequenceNumberBeforeCheckpointing, String regionName, RecordsFetcherFactory recordsFetcherFactory) { + + + this(applicationName, streamName, kinesisEndpoint, dynamoDBEndpoint, initialPositionInStream, kinesisCredentialsProvider, + dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId, maxRecords, idleTimeBetweenReadsInMillis, + callProcessRecordsEvenForEmptyRecordList, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry, + kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig, taskBackoffTimeMillis, metricsBufferTimeMillis, + metricsMaxQueueSize, validateSequenceNumberBeforeCheckpointing, regionName, 0, DEFAULT_DDB_BILLING_MODE, + recordsFetcherFactory, DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS, DEFAULT_COMPLETED_LEASE_CLEANUP_THRESHOLD_MILLIS, + DEFAULT_GARBAGE_LEASE_CLEANUP_THRESHOLD_MILLIS); + } + + /** + * @param applicationName Name of the Kinesis application + * By default the application name is included in the user agent string used to make AWS requests. This + * can assist with troubleshooting (e.g. distinguish requests made by separate applications). + * @param streamName Name of the Kinesis stream + * @param kinesisEndpoint Kinesis endpoint + * @param dynamoDBEndpoint DynamoDB endpoint + * @param initialPositionInStream One of LATEST or TRIM_HORIZON. The KinesisClientLibrary will start fetching + * records from that location in the stream when an application starts up for the first time and there + * are no checkpoints. If there are checkpoints, then we start from the checkpoint position. + * @param kinesisCredentialsProvider Provides credentials used to access Kinesis + * @param dynamoDBCredentialsProvider Provides credentials used to access DynamoDB + * @param cloudWatchCredentialsProvider Provides credentials used to access CloudWatch + * @param failoverTimeMillis Lease duration (leases not renewed within this period will be claimed by others) + * @param workerId Used to distinguish different workers/processes of a Kinesis application + * @param maxRecords Max records to read per Kinesis getRecords() call + * @param idleTimeBetweenReadsInMillis Idle time between calls to fetch data from Kinesis + * @param callProcessRecordsEvenForEmptyRecordList Call the IRecordProcessor::processRecords() API even if + * GetRecords returned an empty record list. + * @param parentShardPollIntervalMillis Wait for this long between polls to check if parent shards are done + * @param shardSyncIntervalMillis Time between tasks to sync leases and Kinesis shards + * @param cleanupTerminatedShardsBeforeExpiry Clean up shards we've finished processing (don't wait for expiration + * in Kinesis) + * @param kinesisClientConfig Client Configuration used by Kinesis client + * @param dynamoDBClientConfig Client Configuration used by DynamoDB client + * @param cloudWatchClientConfig Client Configuration used by CloudWatch client + * @param taskBackoffTimeMillis Backoff period when tasks encounter an exception + * @param metricsBufferTimeMillis Metrics are buffered for at most this long before publishing to CloudWatch + * @param metricsMaxQueueSize Max number of metrics to buffer before publishing to CloudWatch + * @param validateSequenceNumberBeforeCheckpointing whether KCL should validate client provided sequence numbers + * with a call to Amazon Kinesis before checkpointing for calls to + * {@link RecordProcessorCheckpointer#checkpoint(String)} + * @param regionName The region name for the service + * @param shutdownGraceMillis Time before gracefully shutdown forcefully terminates + * @param billingMode The DDB Billing mode to set for lease table creation. + * @param recordsFetcherFactory Factory to create the records fetcher to retrieve data from Kinesis for a given shard. + * @param leaseCleanupIntervalMillis Rate at which to run lease cleanup thread in + * {@link com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager} + * @param completedLeaseCleanupThresholdMillis Threshold in millis at which to check if there are any completed leases + * (leases for shards which have been closed as a result of a resharding operation) that need to be cleaned up. + * @param garbageLeaseCleanupThresholdMillis Threshold in millis at which to check if there are any garbage leases + * (leases for shards which no longer exist in the stream) that need to be cleaned up. + */ + public KinesisClientLibConfiguration(String applicationName, + String streamName, + String kinesisEndpoint, + String dynamoDBEndpoint, + InitialPositionInStream initialPositionInStream, + AWSCredentialsProvider kinesisCredentialsProvider, + AWSCredentialsProvider dynamoDBCredentialsProvider, + AWSCredentialsProvider cloudWatchCredentialsProvider, + long failoverTimeMillis, + String workerId, + int maxRecords, + long idleTimeBetweenReadsInMillis, + boolean callProcessRecordsEvenForEmptyRecordList, + long parentShardPollIntervalMillis, + long shardSyncIntervalMillis, + boolean cleanupTerminatedShardsBeforeExpiry, + ClientConfiguration kinesisClientConfig, + ClientConfiguration dynamoDBClientConfig, + ClientConfiguration cloudWatchClientConfig, + long taskBackoffTimeMillis, + long metricsBufferTimeMillis, + int metricsMaxQueueSize, + boolean validateSequenceNumberBeforeCheckpointing, + String regionName, + long shutdownGraceMillis, + BillingMode billingMode, + RecordsFetcherFactory recordsFetcherFactory, + long leaseCleanupIntervalMillis, + long completedLeaseCleanupThresholdMillis, + long garbageLeaseCleanupThresholdMillis) { + // Check following values are greater than zero checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis); checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis); @@ -627,7 +698,12 @@ public class KinesisClientLibConfiguration { this.shardSyncStrategyType = DEFAULT_SHARD_SYNC_STRATEGY_TYPE; this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; this.recordsFetcherFactory = recordsFetcherFactory; + this.leaseCleanupIntervalMillis = leaseCleanupIntervalMillis; + this.completedLeaseCleanupThresholdMillis = completedLeaseCleanupThresholdMillis; + this.garbageLeaseCleanupThresholdMillis = garbageLeaseCleanupThresholdMillis; this.shutdownGraceMillis = shutdownGraceMillis; + this.billingMode = billingMode; + } // Check if value is positive, otherwise throw an exception @@ -836,6 +912,29 @@ public class KinesisClientLibConfiguration { return cleanupLeasesUponShardCompletion; } + /** + * @return Interval in millis at which to run lease cleanup thread in {@link com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager} + */ + public long leaseCleanupIntervalMillis() { + return leaseCleanupIntervalMillis; + } + + /** + * @return Interval in millis at which to check if there are any completed leases (leases for shards which have been + * closed as a result of a resharding operation) that need to be cleaned up. + */ + public long completedLeaseCleanupThresholdMillis() { + return completedLeaseCleanupThresholdMillis; + } + + /** + * @return Interval in millis at which to check if there are any garbage leases (leases for shards which no longer + * exist in the stream) that need to be cleaned up. + */ + public long garbageLeaseCleanupThresholdMillis() { + return garbageLeaseCleanupThresholdMillis; + } + /** * @return true if we should ignore child shards which have open parents */ @@ -1476,4 +1575,39 @@ public class KinesisClientLibConfiguration { this.maxInitializationAttempts = maxInitializationAttempts; return this; } + + /** + * @param leaseCleanupIntervalMillis Rate at which to run lease cleanup thread in + * {@link com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager} + * @return + */ + public KinesisClientLibConfiguration withLeaseCleanupIntervalMillis(long leaseCleanupIntervalMillis) { + checkIsValuePositive("leaseCleanupIntervalMillis", leaseCleanupIntervalMillis); + this.leaseCleanupIntervalMillis = leaseCleanupIntervalMillis; + return this; + } + + /** + * Threshold in millis at which to check if there are any completed leases (leases for shards which have been + * closed as a result of a resharding operation) that need to be cleaned up. + * @param completedLeaseCleanupThresholdMillis + * @return + */ + public KinesisClientLibConfiguration withCompletedLeaseCleanupThresholdMillis(long completedLeaseCleanupThresholdMillis) { + checkIsValuePositive("completedLeaseCleanupThresholdMillis", completedLeaseCleanupThresholdMillis); + this.completedLeaseCleanupThresholdMillis = completedLeaseCleanupThresholdMillis; + return this; + } + + /** + * Threshold in millis at which to check if there are any garbage leases (leases for shards which no longer exist + * in the stream) that need to be cleaned up. + * @param garbageLeaseCleanupThresholdMillis + * @return + */ + public KinesisClientLibConfiguration withGarbageLeaseCleanupThresholdMillis(long garbageLeaseCleanupThresholdMillis) { + checkIsValuePositive("garbageLeaseCleanupThresholdMillis", garbageLeaseCleanupThresholdMillis); + this.garbageLeaseCleanupThresholdMillis = garbageLeaseCleanupThresholdMillis; + return this; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java index c716afa1..ae4e321d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java @@ -17,10 +17,8 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.Collections; import java.util.Date; import java.util.List; -import java.util.Set; import com.amazonaws.SdkClientException; -import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.model.ChildShard; import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java index 31d8d998..514cfb8c 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisLeaseCleanupValidator.java @@ -10,6 +10,7 @@ import java.util.Set; /** * Represents the class that decides if a lease is eligible for cleanup. */ +@Deprecated class KinesisLeaseCleanupValidator implements LeaseCleanupValidator { private static final Log LOG = LogFactory.getLog(KinesisLeaseCleanupValidator.class); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index 08543230..1ce1175a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -17,8 +17,6 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.io.Serializable; import java.math.BigInteger; import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -26,8 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import com.amazonaws.services.kinesis.leases.impl.Lease; -import com.amazonaws.services.kinesis.leases.impl.LeaseManager; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.ShardFilter; import com.amazonaws.services.kinesis.model.ShardFilterType; @@ -49,8 +45,6 @@ import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.amazonaws.services.kinesis.model.Shard; -import javax.annotation.Nullable; - /** * 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). @@ -67,11 +61,10 @@ class KinesisShardSyncer implements ShardSyncer { } synchronized void bootstrapShardLeases(IKinesisProxy kinesisProxy, ILeaseManager leaseManager, - InitialPositionInStreamExtended initialPositionInStream, boolean cleanupLeasesOfCompletedShards, - boolean ignoreUnexpectedChildShards) - throws DependencyException, InvalidStateException, ProvisionedThroughputException, - KinesisClientLibIOException { - syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, + InitialPositionInStreamExtended initialPositionInStream, + boolean ignoreUnexpectedChildShards) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, ignoreUnexpectedChildShards); } @@ -94,7 +87,7 @@ class KinesisShardSyncer implements ShardSyncer { boolean ignoreUnexpectedChildShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards); + syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, ignoreUnexpectedChildShards); } /** @@ -117,7 +110,7 @@ class KinesisShardSyncer implements ShardSyncer { boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards, List latestShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, + syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, ignoreUnexpectedChildShards, latestShards, leaseManager.isLeaseTableEmpty()); } @@ -127,7 +120,6 @@ class KinesisShardSyncer implements ShardSyncer { * @param kinesisProxy * @param leaseManager * @param initialPosition - * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards * @throws DependencyException * @throws InvalidStateException @@ -135,10 +127,10 @@ class KinesisShardSyncer implements ShardSyncer { * @throws KinesisClientLibIOException */ private synchronized void syncShardLeases(IKinesisProxy kinesisProxy, - ILeaseManager leaseManager, InitialPositionInStreamExtended initialPosition, - boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards) - throws DependencyException, InvalidStateException, ProvisionedThroughputException, - KinesisClientLibIOException { + ILeaseManager leaseManager, + InitialPositionInStreamExtended initialPosition, + boolean ignoreUnexpectedChildShards) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { // In the case where the lease table is empty, we want to synchronize the minimal amount of shards possible // based on the given initial position. @@ -148,7 +140,7 @@ class KinesisShardSyncer implements ShardSyncer { ? getShardListAtInitialPosition(kinesisProxy, initialPosition) : getCompleteShardList(kinesisProxy); - syncShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, + syncShardLeases(kinesisProxy, leaseManager, initialPosition, ignoreUnexpectedChildShards, latestShards, isLeaseTableEmpty); } @@ -158,7 +150,6 @@ class KinesisShardSyncer implements ShardSyncer { * @param kinesisProxy * @param leaseManager * @param initialPosition - * @param cleanupLeasesOfCompletedShards * @param ignoreUnexpectedChildShards * @param latestShards latest snapshot of shards to reuse * @throws DependencyException @@ -170,7 +161,6 @@ class KinesisShardSyncer implements ShardSyncer { private synchronized void syncShardLeases(IKinesisProxy kinesisProxy, ILeaseManager leaseManager, InitialPositionInStreamExtended initialPosition, - boolean cleanupLeasesOfCompletedShards, boolean ignoreUnexpectedChildShards, List latestShards, boolean isLeaseTableEmpty) @@ -218,11 +208,6 @@ class KinesisShardSyncer implements ShardSyncer { trackedLeases.addAll(currentLeases); } trackedLeases.addAll(newLeasesToCreate); - cleanupGarbageLeases(shards, trackedLeases, kinesisProxy, leaseManager); - if (cleanupLeasesOfCompletedShards) { - cleanupLeasesOfFinishedShards(currentLeases, shardIdToShardMap, shardIdToChildShardIdsMap, trackedLeases, - leaseManager); - } } // CHECKSTYLE:ON CyclomaticComplexity @@ -613,150 +598,6 @@ class KinesisShardSyncer implements ShardSyncer { return parentShardIds; } - /** - * Delete leases corresponding to shards that no longer exist in the stream. - * Current scheme: Delete a lease if: - * * the corresponding shard is not present in the list of Kinesis shards, AND - * * the parentShardIds listed in the lease are also not present in the list of Kinesis shards. - * @param shards List of all Kinesis shards (assumed to be a consistent snapshot - when stream is in Active state). - * @param trackedLeases List of - * @param kinesisProxy Kinesis proxy (used to get shard list) - * @param leaseManager - * @throws KinesisClientLibIOException Thrown if we couldn't get a fresh shard list from Kinesis. - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - */ - private void cleanupGarbageLeases(List shards, List trackedLeases, - IKinesisProxy kinesisProxy, ILeaseManager leaseManager) - throws KinesisClientLibIOException, DependencyException, InvalidStateException, - ProvisionedThroughputException { - Set kinesisShards = new HashSet<>(); - for (Shard shard : shards) { - kinesisShards.add(shard.getShardId()); - } - - // Check if there are leases for non-existent shards - List garbageLeases = new ArrayList<>(); - for (KinesisClientLease lease : trackedLeases) { - if (leaseCleanupValidator.isCandidateForCleanup(lease, kinesisShards)) { - garbageLeases.add(lease); - } - } - - if (!garbageLeases.isEmpty()) { - LOG.info("Found " + garbageLeases.size() + " candidate leases for cleanup. Refreshing list of" - + " Kinesis shards to pick up recent/latest shards"); - List currentShardList = getCompleteShardList(kinesisProxy); - Set currentKinesisShardIds = new HashSet<>(); - for (Shard shard : currentShardList) { - currentKinesisShardIds.add(shard.getShardId()); - } - - for (KinesisClientLease lease : garbageLeases) { - if (leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds)) { - LOG.info("Deleting lease for shard " + lease.getLeaseKey() - + " as it is not present in Kinesis stream."); - leaseManager.deleteLease(lease); - } - } - } - - } - - /** - * Private helper method. - * Clean up leases for shards that meet the following criteria: - * a/ the shard has been fully processed (checkpoint is set to SHARD_END) - * b/ we've begun processing all the child shards: we have leases for all child shards and their checkpoint is not - * TRIM_HORIZON. - * - * @param currentLeases List of leases we evaluate for clean up - * @param shardIdToShardMap Map of shardId->Shard (assumed to include all Kinesis shards) - * @param shardIdToChildShardIdsMap Map of shardId->childShardIds (assumed to include all Kinesis shards) - * @param trackedLeases List of all leases we are tracking. - * @param leaseManager Lease manager (will be used to delete leases) - * @throws DependencyException - * @throws InvalidStateException - * @throws ProvisionedThroughputException - * @throws KinesisClientLibIOException - */ - private synchronized void cleanupLeasesOfFinishedShards(Collection currentLeases, - Map shardIdToShardMap, Map> shardIdToChildShardIdsMap, - List trackedLeases, ILeaseManager leaseManager) - throws DependencyException, InvalidStateException, ProvisionedThroughputException, - KinesisClientLibIOException { - Set shardIdsOfClosedShards = new HashSet<>(); - List leasesOfClosedShards = new ArrayList<>(); - for (KinesisClientLease lease : currentLeases) { - if (lease.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END)) { - shardIdsOfClosedShards.add(lease.getLeaseKey()); - leasesOfClosedShards.add(lease); - } - } - - if (!leasesOfClosedShards.isEmpty()) { - assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, shardIdsOfClosedShards); - Comparator startingSequenceNumberComparator = new StartingSequenceNumberAndShardIdBasedComparator( - shardIdToShardMap); - Collections.sort(leasesOfClosedShards, startingSequenceNumberComparator); - Map trackedLeaseMap = constructShardIdToKCLLeaseMap(trackedLeases); - - for (KinesisClientLease leaseOfClosedShard : leasesOfClosedShards) { - String closedShardId = leaseOfClosedShard.getLeaseKey(); - Set childShardIds = shardIdToChildShardIdsMap.get(closedShardId); - if ((closedShardId != null) && (childShardIds != null) && (!childShardIds.isEmpty())) { - cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - } - } - } - } - - /** - * Delete lease for the closed shard. Rules for deletion are: - * a/ the checkpoint for the closed shard is SHARD_END, - * b/ there are leases for all the childShardIds and their checkpoint is NOT TRIM_HORIZON - * Note: This method has package level access solely for testing purposes. - * - * @param closedShardId Identifies the closed shard - * @param childShardIds ShardIds of children of the closed shard - * @param trackedLeases shardId->KinesisClientLease map with all leases we are tracking (should not be null) - * @param leaseManager - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - */ - synchronized void cleanupLeaseForClosedShard(String closedShardId, Set childShardIds, - Map trackedLeases, ILeaseManager leaseManager) - throws DependencyException, InvalidStateException, ProvisionedThroughputException { - KinesisClientLease leaseForClosedShard = trackedLeases.get(closedShardId); - List childShardLeases = new ArrayList<>(); - - for (String childShardId : childShardIds) { - KinesisClientLease childLease = trackedLeases.get(childShardId); - if (childLease != null) { - childShardLeases.add(childLease); - } - } - - if ((leaseForClosedShard != null) && (leaseForClosedShard.getCheckpoint() - .equals(ExtendedSequenceNumber.SHARD_END)) && (childShardLeases.size() == childShardIds.size())) { - boolean okayToDelete = true; - for (KinesisClientLease lease : childShardLeases) { - if (lease.getCheckpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON)) { - okayToDelete = false; - break; - } - } - - if (okayToDelete) { - LOG.info("Deleting lease for shard " + leaseForClosedShard.getLeaseKey() - + " as it has been completely processed and processing of child shards has begun."); - leaseManager.deleteLease(leaseForClosedShard); - } - } - } - /** * Helper method to create a new KinesisClientLease POJO for a shard. * Note: Package level access only for testing purposes diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java index afb37112..7e07587a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/LeaseCleanupValidator.java @@ -8,6 +8,7 @@ import java.util.Set; /** * Represents the class that decides if a lease is eligible for cleanup. */ +@Deprecated public interface LeaseCleanupValidator { /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index f5513d3e..71f8a6bc 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -18,9 +18,11 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.List; import java.util.Optional; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; +import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; @@ -55,6 +57,7 @@ class ShardConsumer { private final IMetricsFactory metricsFactory; private final KinesisClientLibLeaseCoordinator leaseCoordinator; private ICheckpoint checkpoint; + private LeaseCleanupManager leaseCleanupManager; // Backoff time when polling to check if application has finished processing parent shards private final long parentShardPollIntervalMillis; private final boolean cleanupLeasesOfCompletedShards; @@ -112,6 +115,7 @@ class ShardConsumer { * @param shardSyncer shardSyncer instance used to check and create new leases */ // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES + @Deprecated ShardConsumer(ShardInfo shardInfo, StreamConfig streamConfig, ICheckpoint checkpoint, @@ -124,6 +128,7 @@ class ShardConsumer { long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, KinesisClientLibConfiguration config, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) { + this(shardInfo, streamConfig, checkpoint, @@ -156,6 +161,7 @@ class ShardConsumer { * @param shardSyncer shardSyncer instance used to check and create new leases */ // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES + @Deprecated ShardConsumer(ShardInfo shardInfo, StreamConfig streamConfig, ICheckpoint checkpoint, @@ -215,7 +221,9 @@ class ShardConsumer { * @param maxGetRecordsThreadPool max number of threads in the getRecords thread pool * @param config Kinesis library configuration * @param shardSyncer shardSyncer instance used to check and create new leases + * @param leaseCleanupManager used to clean up leases in lease table. */ + @Deprecated ShardConsumer(ShardInfo shardInfo, StreamConfig streamConfig, ICheckpoint checkpoint, @@ -232,6 +240,53 @@ class ShardConsumer { Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool, KinesisClientLibConfiguration config, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy) { + + this(shardInfo, streamConfig, checkpoint, recordProcessor, recordProcessorCheckpointer, leaseCoordinator, + parentShardPollIntervalMillis, cleanupLeasesOfCompletedShards, executorService, metricsFactory, + backoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, kinesisDataFetcher, retryGetRecordsInSeconds, + maxGetRecordsThreadPool, config, shardSyncer, shardSyncStrategy, LeaseCleanupManager.createOrGetInstance(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(), + Executors.newSingleThreadScheduledExecutor(), metricsFactory, config.shouldCleanupLeasesUponShardCompletion(), + config.leaseCleanupIntervalMillis(), config.completedLeaseCleanupThresholdMillis(), + config.garbageLeaseCleanupThresholdMillis(), config.getMaxRecords())); + } + + /** + * @param shardInfo Shard information + * @param streamConfig Stream Config to use + * @param checkpoint Checkpoint tracker + * @param recordProcessor Record processor used to process the data records for the shard + * @param recordProcessorCheckpointer RecordProcessorCheckpointer to use to checkpoint progress + * @param leaseCoordinator Used to manage leases for current worker + * @param parentShardPollIntervalMillis Wait for this long if parent shards are not done (or we get an exception) + * @param cleanupLeasesOfCompletedShards clean up the leases of completed shards + * @param executorService ExecutorService used to execute process tasks for this shard + * @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard + * @param backoffTimeMillis backoff interval when we encounter exceptions + * @param skipShardSyncAtWorkerInitializationIfLeasesExist Skip sync at init if lease exists + * @param kinesisDataFetcher KinesisDataFetcher to fetch data from Kinesis streams. + * @param retryGetRecordsInSeconds time in seconds to wait before the worker retries to get a record + * @param maxGetRecordsThreadPool max number of threads in the getRecords thread pool + * @param config Kinesis library configuration + * @param shardSyncer shardSyncer instance used to check and create new leases + * @param leaseCleanupManager used to clean up leases in lease table. + */ + ShardConsumer(ShardInfo shardInfo, + StreamConfig streamConfig, + ICheckpoint checkpoint, + IRecordProcessor recordProcessor, + RecordProcessorCheckpointer recordProcessorCheckpointer, + KinesisClientLibLeaseCoordinator leaseCoordinator, + long parentShardPollIntervalMillis, + boolean cleanupLeasesOfCompletedShards, + ExecutorService executorService, + IMetricsFactory metricsFactory, + long backoffTimeMillis, + boolean skipShardSyncAtWorkerInitializationIfLeasesExist, + KinesisDataFetcher kinesisDataFetcher, + Optional retryGetRecordsInSeconds, + Optional maxGetRecordsThreadPool, + KinesisClientLibConfiguration config, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy, + LeaseCleanupManager leaseCleanupManager) { this.shardInfo = shardInfo; this.streamConfig = streamConfig; this.checkpoint = checkpoint; @@ -251,6 +306,7 @@ class ShardConsumer { this.getShardInfo().getShardId(), this.metricsFactory, this.config.getMaxRecords()); this.shardSyncer = shardSyncer; this.shardSyncStrategy = shardSyncStrategy; + this.leaseCleanupManager = leaseCleanupManager; } /** @@ -529,4 +585,8 @@ class ShardConsumer { ShardSyncStrategy getShardSyncStrategy() { return shardSyncStrategy; } + + LeaseCleanupManager getLeaseCleanupManager() { + return leaseCleanupManager; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java index 24b30e69..3b04b791 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java @@ -86,7 +86,7 @@ public class ShardInfo { * * @return a list of shardId's that are parents of this shard, or empty if the shard has no parents. */ - protected List getParentShardIds() { + public List getParentShardIds() { return new LinkedList(parentShardIds); } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index 71cf3b9d..300f1ae3 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -14,9 +14,11 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import com.amazonaws.services.kinesis.leases.LeasePendingDeletion; import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; @@ -60,6 +62,7 @@ class ShutdownTask implements ITask { private final ShardSyncer shardSyncer; private final ShardSyncStrategy shardSyncStrategy; private final List childShards; + private final LeaseCleanupManager leaseCleanupManager; /** * Constructor. @@ -76,7 +79,8 @@ class ShutdownTask implements ITask { KinesisClientLibLeaseCoordinator leaseCoordinator, long backoffTimeMillis, GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, - ShardSyncStrategy shardSyncStrategy, List childShards) { + ShardSyncStrategy shardSyncStrategy, List childShards, + LeaseCleanupManager leaseCleanupManager) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -91,6 +95,7 @@ class ShutdownTask implements ITask { this.shardSyncer = shardSyncer; this.shardSyncStrategy = shardSyncStrategy; this.childShards = childShards; + this.leaseCleanupManager = leaseCleanupManager; } /* @@ -153,13 +158,29 @@ class ShutdownTask implements ITask { recordProcessor.shutdown(shutdownInput); ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.getLastCheckpointValue(); + final boolean successfullyCheckpointedShardEnd = lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END); + if (localReason == ShutdownReason.TERMINATE) { - if ((lastCheckpointValue == null) - || (!lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END))) { + if ((lastCheckpointValue == null) || (!successfullyCheckpointedShardEnd)) { throw new IllegalArgumentException("Application didn't checkpoint at end of shard " + shardInfo.getShardId() + ". Application must checkpoint upon shutdown. " + "See IRecordProcessor.shutdown javadocs for more information."); } + + // 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 (successfullyCheckpointedShardEnd || CollectionUtils.isNullOrEmpty(childShards)) { + final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(currentLease, shardInfo); + + if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + } + + //TODO: Add shard end checkpointing here. + } } LOG.debug("Shutting down retrieval strategy."); getRecordsCache.shutdown(); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index f99d24d0..5b4f31e8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -39,6 +39,7 @@ import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector; +import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; import com.amazonaws.services.kinesis.leases.impl.LeaseCoordinator; import com.amazonaws.services.kinesis.leases.impl.LeaseRenewer; import com.amazonaws.services.kinesis.leases.impl.LeaseTaker; @@ -157,6 +158,8 @@ public class Worker implements Runnable { private ShardSyncStrategy shardSyncStrategy; private PeriodicShardSyncManager leaderElectedPeriodicShardSyncManager; + private final LeaseCleanupManager leaseCleanupManager; + /** * Constructor. * @@ -573,6 +576,10 @@ public class Worker implements Runnable { this.workerStateChangeListener = workerStateChangeListener; workerStateChangeListener.onWorkerStateChange(WorkerStateChangeListener.WorkerState.CREATED); createShardSyncStrategy(config.getShardSyncStrategyType(), leaderDecider, periodicShardSyncManager); + this.leaseCleanupManager = LeaseCleanupManager.createOrGetInstance(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(), + Executors.newSingleThreadScheduledExecutor(), metricsFactory, cleanupLeasesUponShardCompletion, + config.leaseCleanupIntervalMillis(), config.completedLeaseCleanupThresholdMillis(), + config.garbageLeaseCleanupThresholdMillis(), config.getMaxRecords()); } /** @@ -726,6 +733,13 @@ public class Worker implements Runnable { } } + 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 the // shard sync in the previous attempt. if (!leaseCoordinator.isRunning()) { @@ -1111,12 +1125,21 @@ public class Worker implements Runnable { } protected ShardConsumer buildConsumer(ShardInfo shardInfo, IRecordProcessorFactory processorFactory) { - IRecordProcessor recordProcessor = processorFactory.createProcessor(); + final IRecordProcessor recordProcessor = processorFactory.createProcessor(); + final RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( + shardInfo, + checkpointTracker, + new SequenceNumberValidator( + streamConfig.getStreamProxy(), + shardInfo.getShardId(), + streamConfig.shouldValidateSequenceNumberBeforeCheckpointing()), + metricsFactory); return new ShardConsumer(shardInfo, streamConfig, checkpointTracker, recordProcessor, + recordProcessorCheckpointer, leaseCoordinator, parentShardPollIntervalMillis, cleanupLeasesUponShardCompletion, @@ -1124,9 +1147,11 @@ public class Worker implements Runnable { metricsFactory, taskBackoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, + new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo), retryGetRecordsInSeconds, maxGetRecordsThreadPool, - config, shardSyncer, shardSyncStrategy); + config, shardSyncer, shardSyncStrategy, + leaseCleanupManager); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/LeasePendingDeletion.java b/src/main/java/com/amazonaws/services/kinesis/leases/LeasePendingDeletion.java new file mode 100644 index 00000000..4f78db3a --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/leases/LeasePendingDeletion.java @@ -0,0 +1,31 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.amazonaws.services.kinesis.leases; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShardInfo; +import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; +import lombok.Value; +import lombok.experimental.Accessors; + +/** + * Helper class for cleaning up leases. + */ +@Accessors(fluent=true) +@Value +public class LeasePendingDeletion { + private final KinesisClientLease lease; + private final ShardInfo shardInfo; +} + diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java new file mode 100644 index 00000000..af02f588 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java @@ -0,0 +1,369 @@ +package com.amazonaws.services.kinesis.leases.impl; + +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShardInfo; +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.leases.LeasePendingDeletion; +import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; +import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; +import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.util.CollectionUtils; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import lombok.AccessLevel; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.NonNull; +import lombok.RequiredArgsConstructor; +import lombok.Value; +import lombok.experimental.Accessors; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Optional; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * Helper class to cleanup of any expired/closed shard leases. It will cleanup leases periodically as defined by + * {@link KinesisClientLibConfiguration#leaseCleanupIntervalMillis()} upon worker shutdown, following a re-shard event or + * a shard expiring from the service. + */ +@RequiredArgsConstructor(access= AccessLevel.PACKAGE) +@EqualsAndHashCode +public class LeaseCleanupManager { + @NonNull + private IKinesisProxy kinesisProxy; + @NonNull + private final ILeaseManager leaseManager; + @NonNull + private final ScheduledExecutorService deletionThreadPool; + @NonNull + private final IMetricsFactory metricsFactory; + private final boolean cleanupLeasesUponShardCompletion; + private final long leaseCleanupIntervalMillis; + private final long completedLeaseCleanupIntervalMillis; + private final long garbageLeaseCleanupIntervalMillis; + private final int maxRecords; + + private final Stopwatch completedLeaseStopwatch = Stopwatch.createUnstarted(); + private final Stopwatch garbageLeaseStopwatch = Stopwatch.createUnstarted(); + private final Queue deletionQueue = new ConcurrentLinkedQueue<>(); + + private static final long INITIAL_DELAY = 0L; + private static final Log LOG = LogFactory.getLog(LeaseCleanupManager.class); + + @Getter + private volatile boolean isRunning = false; + + private static LeaseCleanupManager instance; + + /** + * Factory method to return a singleton instance of {@link LeaseCleanupManager}. + * @param kinesisProxy + * @param leaseManager + * @param deletionThreadPool + * @param metricsFactory + * @param cleanupLeasesUponShardCompletion + * @param leaseCleanupIntervalMillis + * @param completedLeaseCleanupIntervalMillis + * @param garbageLeaseCleanupIntervalMillis + * @param maxRecords + * @return + */ + public static LeaseCleanupManager createOrGetInstance(IKinesisProxy kinesisProxy, ILeaseManager leaseManager, + ScheduledExecutorService deletionThreadPool, IMetricsFactory metricsFactory, + boolean cleanupLeasesUponShardCompletion, long leaseCleanupIntervalMillis, + long completedLeaseCleanupIntervalMillis, long garbageLeaseCleanupIntervalMillis, + int maxRecords) { + if (instance == null) { + instance = new LeaseCleanupManager(kinesisProxy, leaseManager, deletionThreadPool, metricsFactory, cleanupLeasesUponShardCompletion, + leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis, maxRecords); + } + + return instance; + } + + /** + * Starts the lease cleanup thread, which is scheduled periodically as specified by + * {@link LeaseCleanupManager#leaseCleanupIntervalMillis} + */ + public void start() { + LOG.debug("Starting lease cleanup thread."); + isRunning = true; + completedLeaseStopwatch.start(); + garbageLeaseStopwatch.start(); + + deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, + TimeUnit.MILLISECONDS); + } + + /** + * Enqueues a lease for deletion without check for duplicate entry. Use {@link #isEnqueuedForDeletion} + * for checking the duplicate entries. + * @param leasePendingDeletion + */ + public void enqueueForDeletion(LeasePendingDeletion leasePendingDeletion) { + final KinesisClientLease lease = leasePendingDeletion.lease(); + if (lease == null) { + LOG.warn("Cannot enqueue lease " + lease.getLeaseKey() + " for deferred deletion - instance doesn't hold " + + "the lease for that shard."); + } else { + LOG.debug("Enqueuing lease " + lease.getLeaseKey() + " for deferred deletion."); + if (!deletionQueue.add(leasePendingDeletion)) { + LOG.warn("Unable to enqueue lease " + lease.getLeaseKey() + " for deletion."); + } + } + } + + /** + * Check if lease was already enqueued for deletion. + * //TODO: Optimize verifying duplicate entries https://sim.amazon.com/issues/KinesisLTR-597. + * @param leasePendingDeletion + * @return true if enqueued for deletion; false otherwise. + */ + public boolean isEnqueuedForDeletion(LeasePendingDeletion leasePendingDeletion) { + return deletionQueue.contains(leasePendingDeletion); + } + + /** + * Returns how many leases are currently waiting in the queue pending deletion. + * @return number of leases pending deletion. + */ + private int leasesPendingDeletion() { + return deletionQueue.size(); + } + + private boolean timeToCheckForCompletedShard() { + return completedLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= completedLeaseCleanupIntervalMillis; + } + + private boolean timeToCheckForGarbageShard() { + return garbageLeaseStopwatch.elapsed(TimeUnit.MILLISECONDS) >= garbageLeaseCleanupIntervalMillis; + } + + public LeaseCleanupResult cleanupLease(LeasePendingDeletion leasePendingDeletion, + boolean timeToCheckForCompletedShard, boolean timeToCheckForGarbageShard) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final KinesisClientLease lease = leasePendingDeletion.lease(); + final ShardInfo shardInfo = leasePendingDeletion.shardInfo(); + + boolean cleanedUpCompletedLease = false; + boolean cleanedUpGarbageLease = false; + boolean alreadyCheckedForGarbageCollection = false; + boolean wereChildShardsPresent = false; + boolean wasResourceNotFound = false; + + try { + if (cleanupLeasesUponShardCompletion && timeToCheckForCompletedShard) { + final KinesisClientLease leaseFromDDB = leaseManager.getLease(shardInfo.getShardId()); + if(leaseFromDDB != null) { + Set childShardKeys = leaseFromDDB.getChildShardIds(); + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + try { + childShardKeys = getChildShardsFromService(shardInfo); + + if (CollectionUtils.isNullOrEmpty(childShardKeys)) { + LOG.error("No child shards returned from service for shard " + shardInfo.getShardId()); + } else { + wereChildShardsPresent = true; + updateLeaseWithChildShards(leasePendingDeletion, childShardKeys); + } + } catch (ResourceNotFoundException e) { + throw e; + } finally { + alreadyCheckedForGarbageCollection = true; + } + } else { + wereChildShardsPresent = true; + } + try { + cleanedUpCompletedLease = cleanupLeaseForCompletedShard(lease, childShardKeys); + } catch (Exception e) { + // Suppressing the exception here, so that we can attempt for garbage cleanup. + LOG.warn("Unable to cleanup lease for shard " + shardInfo.getShardId()); + } + } else { + LOG.info("Lease not present in lease table while cleaning the shard " + shardInfo.getShardId()); + cleanedUpCompletedLease = true; + } + } + + if (!alreadyCheckedForGarbageCollection && timeToCheckForGarbageShard) { + try { + wereChildShardsPresent = !CollectionUtils + .isNullOrEmpty(getChildShardsFromService(shardInfo)); + } catch (ResourceNotFoundException e) { + throw e; + } + } + } catch (ResourceNotFoundException e) { + wasResourceNotFound = true; + cleanedUpGarbageLease = cleanupLeaseForGarbageShard(lease); + } + + return new LeaseCleanupResult(cleanedUpCompletedLease, cleanedUpGarbageLease, wereChildShardsPresent, + wasResourceNotFound); + } + + private Set getChildShardsFromService(ShardInfo shardInfo) { + final String iterator = kinesisProxy.getIterator(shardInfo.getShardId(), ShardIteratorType.LATEST.toString()); + return kinesisProxy.get(iterator, maxRecords).getChildShards().stream().map(c -> c.getShardId()).collect(Collectors.toSet()); + } + + + // A lease that ended with SHARD_END from ResourceNotFoundException is safe to delete if it no longer exists in the + // stream (known explicitly from ResourceNotFound being thrown when processing this shard), + private boolean cleanupLeaseForGarbageShard(KinesisClientLease lease) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + LOG.info("Deleting lease " + lease.getLeaseKey() + " as it is not present in the stream."); + leaseManager.deleteLease(lease); + return true; + } + + private boolean allParentShardLeasesDeleted(KinesisClientLease lease) throws DependencyException, ProvisionedThroughputException, InvalidStateException { + for (String parentShard : lease.getParentShardIds()) { + final KinesisClientLease parentLease = leaseManager.getLease(parentShard); + + if (parentLease != null) { + LOG.warn("Lease " + lease.getLeaseKey() + " has a parent lease " + parentLease.getLeaseKey() + + " which is still present in the lease table, skipping deletion for this lease."); + return false; + } + } + return true; + } + + // We should only be deleting the current shard's lease if + // 1. All of its children are currently being processed, i.e their checkpoint is not TRIM_HORIZON or AT_TIMESTAMP. + // 2. Its parent shard lease(s) have already been deleted. + private boolean cleanupLeaseForCompletedShard(KinesisClientLease lease, Set childShardLeaseKeys) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, IllegalStateException { + final Set processedChildShardLeaseKeys = new HashSet<>(); + + for (String childShardLeaseKey : childShardLeaseKeys) { + final KinesisClientLease childShardLease = Optional.ofNullable( + leaseManager.getLease(childShardLeaseKey)) + .orElseThrow(() -> new IllegalStateException( + "Child lease " + childShardLeaseKey + " for completed shard not found in " + + "lease table - not cleaning up lease " + lease)); + + if (!childShardLease.getCheckpoint().equals(ExtendedSequenceNumber.TRIM_HORIZON) && !childShardLease + .getCheckpoint().equals(ExtendedSequenceNumber.AT_TIMESTAMP)) { + processedChildShardLeaseKeys.add(childShardLease.getLeaseKey()); + } + } + + if (!allParentShardLeasesDeleted(lease) || !Objects.equals(childShardLeaseKeys, processedChildShardLeaseKeys)) { + return false; + } + + LOG.info("Deleting lease " + lease.getLeaseKey() + " as it has been completely processed and processing of child shard(s) has begun."); + leaseManager.deleteLease(lease); + + return true; + } + + private void updateLeaseWithChildShards(LeasePendingDeletion leasePendingDeletion, Set childShardKeys) + throws DependencyException, ProvisionedThroughputException, InvalidStateException { + final KinesisClientLease updatedLease = leasePendingDeletion.lease(); + updatedLease.setChildShardIds(childShardKeys); + + leaseManager.updateLease(updatedLease); + } + + @VisibleForTesting + void cleanupLeases() { + LOG.info("Number of pending leases to clean before the scan : " + leasesPendingDeletion()); + if (deletionQueue.isEmpty()) { + LOG.debug("No leases pending deletion."); + } else if (timeToCheckForCompletedShard() | timeToCheckForGarbageShard()) { + final Queue failedDeletions = new ConcurrentLinkedQueue<>(); + boolean completedLeaseCleanedUp = false; + boolean garbageLeaseCleanedUp = false; + + LOG.debug("Attempting to clean up " + deletionQueue.size() + " lease(s)."); + + while (!deletionQueue.isEmpty()) { + final LeasePendingDeletion leasePendingDeletion = deletionQueue.poll(); + final String leaseKey = leasePendingDeletion.lease().getLeaseKey(); + boolean deletionSucceeded = false; + try { + final LeaseCleanupResult leaseCleanupResult = cleanupLease(leasePendingDeletion, + timeToCheckForCompletedShard(), timeToCheckForGarbageShard()); + completedLeaseCleanedUp |= leaseCleanupResult.cleanedUpCompletedLease(); + garbageLeaseCleanedUp |= leaseCleanupResult.cleanedUpGarbageLease(); + + if (leaseCleanupResult.leaseCleanedUp()) { + LOG.debug("Successfully cleaned up lease " + leaseKey); + deletionSucceeded = true; + } else { + LOG.warn("Unable to clean up lease " + leaseKey + " due to " + leaseCleanupResult); + } + } catch (Exception e) { + LOG.error("Failed to cleanup lease " + leaseKey + ". Will re-enqueue for deletion and retry on next " + + "scheduled execution.", e); + } + if (!deletionSucceeded) { + LOG.debug("Did not cleanup lease " + leaseKey + ". Re-enqueueing for deletion."); + failedDeletions.add(leasePendingDeletion); + } + } + if (completedLeaseCleanedUp) { + LOG.debug("At least one completed lease was cleaned up - restarting interval"); + completedLeaseStopwatch.reset().start(); + } + if (garbageLeaseCleanedUp) { + LOG.debug("At least one garbage lease was cleaned up - restarting interval"); + garbageLeaseStopwatch.reset().start(); + } + deletionQueue.addAll(failedDeletions); + + LOG.info("Number of pending leases to clean after the scan : " + leasesPendingDeletion()); + } + } + + private class LeaseCleanupThread implements Runnable { + @Override + public void run() { + cleanupLeases(); + } + } + + @Value + @Accessors(fluent=true) + public static class LeaseCleanupResult { + boolean cleanedUpCompletedLease; + boolean cleanedUpGarbageLease; + boolean wereChildShardsPresent; + boolean wasResourceNotFound; + + public boolean leaseCleanedUp() { + return cleanedUpCompletedLease | cleanedUpGarbageLease; + } + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java index d9160f0f..6a5e76b9 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java @@ -124,7 +124,7 @@ public class ConsumerStatesTest { assertThat(state.successTransition(), equalTo(ShardConsumerState.INITIALIZING.getConsumerState())); for (ShutdownReason shutdownReason : ShutdownReason.values()) { assertThat(state.shutdownTransition(shutdownReason), - equalTo(ShardConsumerState.SHUTDOWN_COMPLETE.getConsumerState())); + equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState())); } assertThat(state.getState(), equalTo(ShardConsumerState.WAITING_ON_PARENT_SHARDS)); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index f040c6a6..67bf3697 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -54,6 +54,8 @@ import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; +import com.amazonaws.services.kinesis.leases.impl.LeaseManager; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.hamcrest.Description; @@ -138,6 +140,7 @@ public class ShardConsumerTest { recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.empty()); + when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); } /** @@ -475,6 +478,8 @@ public class ShardConsumerTest { when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseManager.getLease(eq(parentShardId))).thenReturn(parentLease); when(parentLease.getCheckpoint()).thenReturn(ExtendedSequenceNumber.TRIM_HORIZON); + when(recordProcessorCheckpointer.getLastCheckpointValue()).thenReturn(ExtendedSequenceNumber.SHARD_END); + when(streamConfig.getStreamProxy()).thenReturn(streamProxy); final ShardConsumer consumer = new ShardConsumer(shardInfo, @@ -507,6 +512,9 @@ public class ShardConsumerTest { assertThat(consumer.getShutdownReason(), equalTo(ShutdownReason.REQUESTED)); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.SHUTTING_DOWN))); + Thread.sleep(50L); + consumer.beginShutdown(); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE))); assertThat(consumer.isShutdown(), is(true)); verify(shutdownNotification, times(1)).shutdownComplete(); @@ -704,19 +712,19 @@ public class ShardConsumerTest { StreamConfig streamConfig = new StreamConfig(fileBasedProxy, - maxRecords, - idleTimeMS, - callProcessRecordsForEmptyRecordList, - skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + maxRecords, + idleTimeMS, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, INITIAL_POSITION_LATEST); ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, null); dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, - new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); + new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), - any(IMetricsFactory.class), anyInt())) + any(IMetricsFactory.class), anyInt())) .thenReturn(getRecordsCache); RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( @@ -732,29 +740,29 @@ public class ShardConsumerTest { ShardConsumer consumer = new ShardConsumer(shardInfo, - streamConfig, - checkpoint, - processor, - recordProcessorCheckpointer, - leaseCoordinator, - parentShardPollIntervalMillis, - cleanupLeasesOfCompletedShards, - executorService, - metricsFactory, - taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, - dataFetcher, - Optional.empty(), - Optional.empty(), - config, - shardSyncer, - shardSyncStrategy); + streamConfig, + checkpoint, + processor, + recordProcessorCheckpointer, + leaseCoordinator, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + dataFetcher, + Optional.empty(), + Optional.empty(), + config, + shardSyncer, + shardSyncStrategy); List parentShardIds = new ArrayList<>(); parentShardIds.add(shardInfo.getShardId()); when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(createLease(shardInfo.getShardId(), - "leaseOwner", - parentShardIds)); + "leaseOwner", + parentShardIds)); when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); @@ -1111,7 +1119,7 @@ public class ShardConsumerTest { //@formatter:off (gets the formatting wrong) private void verifyConsumedRecords(List expectedRecords, - List actualRecords) { + List actualRecords) { //@formatter:on assertThat(actualRecords.size(), is(equalTo(expectedRecords.size()))); ListIterator expectedIter = expectedRecords.listIterator(); @@ -1141,7 +1149,7 @@ public class ShardConsumerTest { } Matcher initializationInputMatcher(final ExtendedSequenceNumber checkpoint, - final ExtendedSequenceNumber pendingCheckpoint) { + final ExtendedSequenceNumber pendingCheckpoint) { return new TypeSafeMatcher() { @Override protected boolean matchesSafely(InitializationInput item) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardObjectHelper.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardObjectHelper.java index a155f5c4..b80064af 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardObjectHelper.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardObjectHelper.java @@ -25,24 +25,24 @@ import com.amazonaws.services.kinesis.model.Shard; /** * Helper class to create Shard, SequenceRange and related objects. */ -class ShardObjectHelper { +public class ShardObjectHelper { private static final int EXPONENT = 128; /** * Max value of a sequence number (2^128 -1). Useful for defining sequence number range for a shard. */ - static final String MAX_SEQUENCE_NUMBER = new BigInteger("2").pow(EXPONENT).subtract(BigInteger.ONE).toString(); + public static final String MAX_SEQUENCE_NUMBER = new BigInteger("2").pow(EXPONENT).subtract(BigInteger.ONE).toString(); /** * Min value of a sequence number (0). Useful for defining sequence number range for a shard. */ - static final String MIN_SEQUENCE_NUMBER = BigInteger.ZERO.toString(); + public static final String MIN_SEQUENCE_NUMBER = BigInteger.ZERO.toString(); /** * Max value of a hash key (2^128 -1). Useful for defining hash key range for a shard. */ - static final String MAX_HASH_KEY = new BigInteger("2").pow(EXPONENT).subtract(BigInteger.ONE).toString(); + public static final String MAX_HASH_KEY = new BigInteger("2").pow(EXPONENT).subtract(BigInteger.ONE).toString(); /** * Min value of a hash key (0). Useful for defining sequence number range for a shard. @@ -63,7 +63,7 @@ class ShardObjectHelper { * @param sequenceNumberRange * @return */ - static Shard newShard(String shardId, + public static Shard newShard(String shardId, String parentShardId, String adjacentParentShardId, SequenceNumberRange sequenceNumberRange) { @@ -78,7 +78,7 @@ class ShardObjectHelper { * @param hashKeyRange * @return */ - static Shard newShard(String shardId, + public static Shard newShard(String shardId, String parentShardId, String adjacentParentShardId, SequenceNumberRange sequenceNumberRange, @@ -98,7 +98,7 @@ class ShardObjectHelper { * @param endingSequenceNumber * @return */ - static SequenceNumberRange newSequenceNumberRange(String startingSequenceNumber, String endingSequenceNumber) { + public static SequenceNumberRange newSequenceNumberRange(String startingSequenceNumber, String endingSequenceNumber) { SequenceNumberRange range = new SequenceNumberRange(); range.setStartingSequenceNumber(startingSequenceNumber); range.setEndingSequenceNumber(endingSequenceNumber); @@ -110,14 +110,14 @@ class ShardObjectHelper { * @param endingHashKey * @return */ - static HashKeyRange newHashKeyRange(String startingHashKey, String endingHashKey) { + public static HashKeyRange newHashKeyRange(String startingHashKey, String endingHashKey) { HashKeyRange range = new HashKeyRange(); range.setStartingHashKey(startingHashKey); range.setEndingHashKey(endingHashKey); return range; } - static List getParentShardIds(Shard shard) { + public static List getParentShardIds(Shard shard) { List parentShardIds = new ArrayList<>(2); if (shard.getAdjacentParentShardId() != null) { parentShardIds.add(shard.getAdjacentParentShardId()); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index 208d6448..e2cc578c 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -420,6 +420,7 @@ public class ShardSyncerTest { cleanupLeasesOfCompletedShards, true, shards); List newLeases = leaseManager.listLeases(); Set expectedLeaseShardIds = new HashSet(); + expectedLeaseShardIds.add("shardId-1000"); // dummy lease will still be in the table. expectedLeaseShardIds.add("shardId-4"); expectedLeaseShardIds.add("shardId-5"); expectedLeaseShardIds.add("shardId-8"); @@ -691,31 +692,6 @@ public class ShardSyncerTest { dataFile.delete(); } - /** - * Test bootstrapShardLeases() - cleanup garbage leases. - * - * @throws ProvisionedThroughputException - * @throws InvalidStateException - * @throws DependencyException - * @throws IOException - * @throws KinesisClientLibIOException - */ - @Test - public final void testBootstrapShardLeasesCleanupGarbage() - throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException, - KinesisClientLibIOException { - String garbageShardId = "shardId-garbage-001"; - KinesisClientLease garbageLease = shardSyncer.newKCLLease(ShardObjectHelper.newShard(garbageShardId, - null, - null, - ShardObjectHelper.newSequenceNumberRange("101", null))); - garbageLease.setCheckpoint(new ExtendedSequenceNumber("999")); - leaseManager.createLeaseIfNotExists(garbageLease); - Assert.assertEquals(garbageShardId, leaseManager.getLease(garbageShardId).getLeaseKey()); - testBootstrapShardLeasesAtStartingPosition(INITIAL_POSITION_LATEST); - Assert.assertNull(leaseManager.getLease(garbageShardId)); - } - private void testBootstrapShardLeasesAtStartingPosition(InitialPositionInStreamExtended initialPosition) throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException, KinesisClientLibIOException { @@ -730,7 +706,7 @@ public class ShardSyncerTest { dataFile.deleteOnExit(); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); - shardSyncer.bootstrapShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, + shardSyncer.bootstrapShardLeases(kinesisProxy, leaseManager, initialPosition, false); List newLeases = leaseManager.listLeases(); Assert.assertEquals(2, newLeases.size()); @@ -2267,81 +2243,6 @@ public class ShardSyncerTest { Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds)); } - /** - * Test cleanup of lease for a shard that has been fully processed (and processing of child shards has begun). - * - * @throws DependencyException - * @throws InvalidStateException - * @throws ProvisionedThroughputException - */ - @Test - public final void testCleanupLeaseForClosedShard() - throws DependencyException, InvalidStateException, ProvisionedThroughputException { - String closedShardId = "shardId-2"; - KinesisClientLease leaseForClosedShard = newLease(closedShardId); - leaseForClosedShard.setCheckpoint(new ExtendedSequenceNumber("1234")); - leaseManager.createLeaseIfNotExists(leaseForClosedShard); - - Set childShardIds = new HashSet<>(); - List trackedLeases = new ArrayList<>(); - Set parentShardIds = new HashSet<>(); - parentShardIds.add(closedShardId); - String childShardId1 = "shardId-5"; - KinesisClientLease childLease1 = newLease(childShardId1); - childLease1.setParentShardIds(parentShardIds); - childLease1.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - String childShardId2 = "shardId-7"; - KinesisClientLease childLease2 = newLease(childShardId2); - childLease2.setParentShardIds(parentShardIds); - childLease2.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); - Map trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); - - // empty list of leases - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNotNull(leaseManager.getLease(closedShardId)); - - // closed shard has not been fully processed yet (checkpoint != SHARD_END) - trackedLeases.add(leaseForClosedShard); - trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNotNull(leaseManager.getLease(closedShardId)); - - // closed shard has been fully processed yet (checkpoint == SHARD_END) - leaseForClosedShard.setCheckpoint(ExtendedSequenceNumber.SHARD_END); - leaseManager.updateLease(leaseForClosedShard); - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNull(leaseManager.getLease(closedShardId)); - - // lease for only one child exists - childShardIds.add(childShardId1); - childShardIds.add(childShardId2); - leaseManager.createLeaseIfNotExists(leaseForClosedShard); - leaseManager.createLeaseIfNotExists(childLease1); - trackedLeases.add(childLease1); - trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNotNull(leaseManager.getLease(closedShardId)); - - // leases for both children exists, but they are both at TRIM_HORIZON - leaseManager.createLeaseIfNotExists(childLease2); - trackedLeases.add(childLease2); - trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNotNull(leaseManager.getLease(closedShardId)); - - // leases for both children exists, one is at TRIM_HORIZON - childLease1.setCheckpoint(new ExtendedSequenceNumber("34890")); - leaseManager.updateLease(childLease1); - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNotNull(leaseManager.getLease(closedShardId)); - - // leases for both children exists, NONE of them are at TRIM_HORIZON - childLease2.setCheckpoint(new ExtendedSequenceNumber("43789")); - leaseManager.updateLease(childLease2); - shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); - Assert.assertNull(leaseManager.getLease(closedShardId)); - } - /** * Test we can handle trimmed Kinesis shards (absent from the shard list), and valid closed shards. * diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index cbfdf54a..b47bf70d 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -34,6 +34,7 @@ import java.util.UUID; import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; @@ -86,6 +87,8 @@ public class ShutdownTaskTest { private ILeaseManager leaseManager; @Mock private KinesisClientLibLeaseCoordinator leaseCoordinator; + @Mock + private LeaseCleanupManager leaseCleanupManager; /** * @throws java.lang.Exception @@ -143,7 +146,8 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - constructChildShards()); + constructChildShards(), + leaseCleanupManager); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof IllegalArgumentException); @@ -178,7 +182,8 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - constructChildShards()); + constructChildShards(), + leaseCleanupManager); TaskResult result = task.call(); verify(getRecordsCache).shutdown(); verify(leaseCoordinator).dropLease(any(KinesisClientLease.class)); @@ -205,7 +210,8 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - constructChildShards()); + constructChildShards(), + leaseCleanupManager); TaskResult result = task.call(); verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseCoordinator).updateLease(any(KinesisClientLease.class), any(UUID.class)); @@ -238,7 +244,8 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - Collections.emptyList()); + Collections.emptyList(), + leaseCleanupManager); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); @@ -267,7 +274,8 @@ public class ShutdownTaskTest { getRecordsCache, shardSyncer, shardSyncStrategy, - Collections.emptyList()); + Collections.emptyList(), + leaseCleanupManager); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseCoordinator, never()).updateLease(any(KinesisClientLease.class), any(UUID.class)); @@ -284,7 +292,7 @@ public class ShutdownTaskTest { ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, leaseCoordinator, 0, - getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList()); + getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList(), leaseCleanupManager); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java new file mode 100644 index 00000000..367c0ab0 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java @@ -0,0 +1,289 @@ +/* + * Copyright 2020 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.amazonaws.services.kinesis.leases.impl; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShardInfo; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShardObjectHelper; +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.leases.LeasePendingDeletion; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.model.ChildShard; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyString; +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 String concurrencyToken = "1234"; + private int maxRecords = 1; + + private String getShardId = "getShardId"; + private String splitParent = "splitParent"; + private String mergeParent1 = "mergeParent-1"; + private String mergeParent2 = "mergeParent-2"; + + 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 IMetricsFactory NULL_METRICS_FACTORY = new NullMetricsFactory(); + + @Mock + private LeaseManager leaseManager; + @Mock + private LeaseCoordinator leaseCoordinator; + @Mock + private IKinesisProxy kinesis; + @Mock + private ScheduledExecutorService deletionThreadPool; + + @Before + public void setUp() { + shardInfo = new ShardInfo(getShardId, concurrencyToken, Collections.emptySet(), + ExtendedSequenceNumber.LATEST); + leaseCleanupManager = new LeaseCleanupManager(kinesis, leaseManager, deletionThreadPool, NULL_METRICS_FACTORY, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis, maxRecords); + } + + /** + * 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(kinesis, leaseManager, deletionThreadPool, NULL_METRICS_FACTORY, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis, maxRecords); + + 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 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 KinesisClientLease heldLease = LeaseHelper.createLease(shardInfo.getShardId(), "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 KinesisClientLease heldLease = LeaseHelper.createLease(shardInfo.getShardId(), "leaseOwner", Collections.singleton("parentShardId")); + + cleanupLeasesOfCompletedShards = false; + + leaseCleanupManager = new LeaseCleanupManager(kinesis, leaseManager, deletionThreadPool, NULL_METRICS_FACTORY, + cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis, maxRecords); + + testLeaseDeletedWhenShardDoesNotExist(heldLease); + } + + public void testLeaseDeletedWhenShardDoesNotExist(KinesisClientLease heldLease) throws Exception { + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(heldLease); + when(kinesis.get(anyString(), anyInt())).thenThrow(ResourceNotFoundException.class); + when(kinesis.getIterator(anyString(), anyString())).thenThrow(ResourceNotFoundException.class); + when(leaseManager.getLease(heldLease.getLeaseKey())).thenReturn(heldLease); + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(heldLease, shardInfo)); + leaseCleanupManager.cleanupLeases(); + + verify(leaseManager, times(1)).deleteLease(heldLease); + } + + private void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + ExtendedSequenceNumber extendedSequenceNumber, + int expectedDeletedLeases) throws Exception { + verifyExpectedDeletedLeasesCompletedShardCase(shardInfo, childShards, extendedSequenceNumber, true, expectedDeletedLeases); + } + + private void verifyExpectedDeletedLeasesCompletedShardCase(ShardInfo shardInfo, List childShards, + ExtendedSequenceNumber extendedSequenceNumber, + boolean childShardLeasesPresent, + int expectedDeletedLeases) throws Exception { + + final KinesisClientLease lease = LeaseHelper.createLease(shardInfo.getShardId(), "leaseOwner", shardInfo.getParentShardIds(), + childShards.stream().map(c -> c.getShardId()).collect(Collectors.toSet())); + final List childShardLeases = childShards.stream().map(c -> LeaseHelper.createLease( + c.getShardId(), "leaseOwner", Collections.singleton(shardInfo.getShardId()), + Collections.emptyList(), extendedSequenceNumber)).collect(Collectors.toList()); + + final List parentShardLeases = lease.getParentShardIds().stream().map(p -> + LeaseHelper.createLease(p, "leaseOwner", Collections.emptyList(), + Collections.singleton(shardInfo.getShardId()), extendedSequenceNumber)).collect(Collectors.toList()); + + when(leaseManager.getLease(lease.getLeaseKey())).thenReturn(lease); + for (Lease parentShardLease : parentShardLeases) { + when(leaseManager.getLease(parentShardLease.getLeaseKey())).thenReturn(parentShardLease); + } + if (childShardLeasesPresent) { + for (Lease childShardLease : childShardLeases) { + when(leaseManager.getLease(childShardLease.getLeaseKey())).thenReturn(childShardLease); + } + } + + when(kinesis.getIterator(any(String.class), any(String.class))).thenReturn("123"); + + final GetRecordsResult getRecordsResult = new GetRecordsResult(); + getRecordsResult.setRecords(Collections.emptyList()); + getRecordsResult.setChildShards(childShards); + + when(kinesis.get(any(String.class), any(Integer.class))).thenReturn(getRecordsResult); + + leaseCleanupManager.enqueueForDeletion(new LeasePendingDeletion(lease, shardInfo)); + leaseCleanupManager.cleanupLeases(); + + verify(leaseManager, times(expectedDeletedLeases)).deleteLease(any(Lease.class)); + } + + private List childShardsForSplit() { + final List parentShards = Arrays.asList(splitParent); + + final ChildShard leftChild = new ChildShard(); + leftChild.setShardId("leftChild"); + leftChild.setParentShards(parentShards); + leftChild.setHashKeyRange(ShardObjectHelper.newHashKeyRange("0", "49")); + + final ChildShard rightChild = new ChildShard(); + rightChild.setShardId("rightChild"); + rightChild.setParentShards(parentShards); + rightChild.setHashKeyRange(ShardObjectHelper.newHashKeyRange("50", "99")); + + return Arrays.asList(leftChild, rightChild); + } + + private List childShardsForMerge() { + final List parentShards = Arrays.asList(mergeParent1, mergeParent2); + + final ChildShard child = new ChildShard(); + child.setShardId("onlyChild"); + child.setParentShards(parentShards); + child.setHashKeyRange(ShardObjectHelper.newHashKeyRange("0", "99")); + + return Collections.singletonList(child); + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseHelper.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseHelper.java new file mode 100644 index 00000000..b122d9ff --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/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 com.amazonaws.services.kinesis.leases.impl; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; + +import java.util.Collection; +import java.util.Collections; + +public class LeaseHelper { + + public static KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds) { + return createLease(leaseKey, leaseOwner, parentShardIds, Collections.emptySet(), ExtendedSequenceNumber.LATEST); + } + + public static KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds, Collection childShardIds) { + return createLease(leaseKey, leaseOwner, parentShardIds, childShardIds, ExtendedSequenceNumber.LATEST); + } + + public static KinesisClientLease createLease(String leaseKey, String leaseOwner, Collection parentShardIds, + Collection childShardIds, ExtendedSequenceNumber extendedSequenceNumber) { + KinesisClientLease lease = new KinesisClientLease (); + lease.setLeaseKey(leaseKey); + lease.setLeaseOwner(leaseOwner); + lease.setParentShardIds(parentShardIds); + lease.setChildShardIds(childShardIds); + lease.setCheckpoint(extendedSequenceNumber); + + return lease; + } +} \ No newline at end of file From c3b41c3b5547c3bd665c6852a32e231cdee0812e Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 27 Jul 2020 18:18:32 -0700 Subject: [PATCH 18/43] refactoring shutdownTask --- .../lib/worker/ConsumerStates.java | 3 +- .../lib/worker/ShardConsumer.java | 2 +- .../lib/worker/ShutdownTask.java | 218 ++++++++++-------- .../CustomerApplicationException.java | 24 ++ .../lib/worker/ShardConsumerTest.java | 19 +- .../lib/worker/ShutdownTaskTest.java | 43 ++-- 6 files changed, 197 insertions(+), 112 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/leases/exceptions/CustomerApplicationException.java diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index 5cf55dbf..b5cde2bc 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -532,7 +532,8 @@ class ConsumerStates { consumer.getTaskBackoffTimeMillis(), consumer.getGetRecordsCache(), consumer.getShardSyncer(), consumer.getShardSyncStrategy(), consumer.getChildShards(), - consumer.getLeaseCleanupManager()); + consumer.getLeaseCleanupManager(), + consumer.getMetricsFactory()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 71f8a6bc..10600def 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -54,6 +54,7 @@ class ShardConsumer { private final ExecutorService executorService; private final ShardInfo shardInfo; private final KinesisDataFetcher dataFetcher; + @Getter private final IMetricsFactory metricsFactory; private final KinesisClientLibLeaseCoordinator leaseCoordinator; private ICheckpoint checkpoint; @@ -221,7 +222,6 @@ class ShardConsumer { * @param maxGetRecordsThreadPool max number of threads in the getRecords thread pool * @param config Kinesis library configuration * @param shardSyncer shardSyncer instance used to check and create new leases - * @param leaseCleanupManager used to clean up leases in lease table. */ @Deprecated ShardConsumer(ShardInfo shardInfo, diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index d30ec765..a9ec8d76 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -16,11 +16,13 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import com.amazonaws.services.kinesis.leases.LeasePendingDeletion; import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.BlockedOnParentShardException; +import com.amazonaws.services.kinesis.leases.exceptions.CustomerApplicationException; import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; import com.amazonaws.services.kinesis.leases.impl.UpdateField; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; @@ -37,6 +39,7 @@ import com.google.common.annotations.VisibleForTesting; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.stream.Collectors; @@ -48,6 +51,7 @@ class ShutdownTask implements ITask { private static final Log LOG = LogFactory.getLog(ShutdownTask.class); + 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; @@ -68,6 +72,7 @@ class ShutdownTask implements ITask { private final ShardSyncStrategy shardSyncStrategy; private final List childShards; private final LeaseCleanupManager leaseCleanupManager; + private final IMetricsFactory metricsFactory; /** * Constructor. @@ -85,7 +90,7 @@ class ShutdownTask implements ITask { long backoffTimeMillis, GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy, List childShards, - LeaseCleanupManager leaseCleanupManager) { + LeaseCleanupManager leaseCleanupManager, IMetricsFactory metricsFactory) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -101,6 +106,7 @@ class ShutdownTask implements ITask { this.shardSyncStrategy = shardSyncStrategy; this.childShards = childShards; this.leaseCleanupManager = leaseCleanupManager; + this.metricsFactory = metricsFactory; } /* @@ -111,110 +117,143 @@ class ShutdownTask implements ITask { */ @Override public TaskResult call() { + MetricsHelper.startScope(metricsFactory, SHUTDOWN_TASK_OPERATION); Exception exception; - boolean applicationException = false; try { LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: " - + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); + + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); - ShutdownReason localReason = attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(reason); - - final ShutdownInput shutdownInput = new ShutdownInput() - .withShutdownReason(localReason) - .withCheckpointer(recordProcessorCheckpointer); - final long recordProcessorStartTimeMillis = System.currentTimeMillis(); try { - recordProcessor.shutdown(shutdownInput); - ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.getLastCheckpointValue(); + final long startTime = System.currentTimeMillis(); + final KinesisClientLease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + final Runnable leaseLostAction = () -> takeLeaseLostAction(); - final boolean successfullyCheckpointedShardEnd = lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END); - - if (localReason == ShutdownReason.TERMINATE) { - if ((lastCheckpointValue == null) || (!successfullyCheckpointedShardEnd)) { - throw new IllegalArgumentException("Application didn't checkpoint at end of shard " - + shardInfo.getShardId() + ". Application must checkpoint upon shutdown. " + - "See IRecordProcessor.shutdown javadocs for more information."); - } - - // 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 (successfullyCheckpointedShardEnd || CollectionUtils.isNullOrEmpty(childShards)) { - final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); - final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(currentLease, shardInfo); - - if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { - leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); - } - - //TODO: Add shard end checkpointing here. + if (reason == ShutdownReason.TERMINATE) { + try { + takeShardEndAction(currentShardLease, 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 ZOMBIE reason to allow other worker to take the lease and retry shutting down. + LOG.warn("Lease " + shardInfo.getShardId() + ": Invalid state encountered while shutting down shardConsumer with TERMINATE reason. " + + "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. ", e); + dropLease(currentShardLease); + throwOnApplicationException(leaseLostAction, startTime); } + } else { + throwOnApplicationException(leaseLostAction, startTime); } + LOG.debug("Shutting down retrieval strategy."); getRecordsCache.shutdown(); LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId()); + return new TaskResult(null); } catch (Exception e) { - applicationException = true; - throw e; - } finally { - MetricsHelper.addLatency(RECORD_PROCESSOR_SHUTDOWN_METRIC, recordProcessorStartTimeMillis, - MetricsLevel.SUMMARY); - } + if (e instanceof CustomerApplicationException) { + LOG.error("Shard " + shardInfo.getShardId() + ": Application exception: ", e); + } else { + LOG.error("Shard " + shardInfo.getShardId() + ": Caught exception: ", e); + } - return new TaskResult(null); - } catch (Exception e) { - if (applicationException) { - LOG.error("Application exception. ", e); - } else { - LOG.error("Caught exception: ", e); - } - exception = e; - // backoff if we encounter an exception. - try { - Thread.sleep(this.backoffTimeMillis); - } catch (InterruptedException ie) { - LOG.debug("Interrupted sleep", ie); + exception = e; + // backoff if we encounter an exception. + try { + Thread.sleep(this.backoffTimeMillis); + } catch (InterruptedException ie) { + LOG.debug("Interrupted sleep", ie); + } } + } finally { + MetricsHelper.endScope(); } return new TaskResult(exception); } - private ShutdownReason attemptPersistingChildShardInfoAndOverrideShutdownReasonOnFailure(ShutdownReason originalReason) - throws DependencyException, ProvisionedThroughputException { - ShutdownReason shutdownReason = originalReason; - if(originalReason == ShutdownReason.TERMINATE) { - // For TERMINATE shutdown reason, try to create and persist childShard leases before setting checkpoint. - try { - final KinesisClientLease currentLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); - if (currentLease == null) { - throw new InvalidStateException(shardInfo.getShardId() - + " : Lease not owned by the current worker. Leaving ShardEnd handling to new owner."); - } - if (!CollectionUtils.isNullOrEmpty(childShards)) { - createLeasesForChildShardsIfNotExist(); - updateCurrentLeaseWithChildShards(currentLease); - recordProcessorCheckpointer.setSequenceNumberAtShardEnd( - recordProcessorCheckpointer.getLargestPermittedCheckpointValue()); - recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); - } else { - LOG.warn("Shard " + shardInfo.getShardId() - + ": Shutting down consumer with SHARD_END reason without creating leases for child shards."); - } - } catch (InvalidStateException e) { - // If invalidStateException happens, it indicates we are missing childShard related information. - // In this scenario, we should shutdown the shardConsumer with ZOMBIE reason to allow other worker to take the lease and retry getting - // childShard information in the processTask. - shutdownReason = ShutdownReason.ZOMBIE; - dropLease(); - LOG.warn("Shard " + shardInfo.getShardId() + ": Exception happened while shutting down shardConsumer with TERMINATE reason. " + - "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. Exception: ", e); - } - + // Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup. + private void takeShardEndAction(KinesisClientLease currentShardLease, long startTime) + throws InvalidStateException, DependencyException, ProvisionedThroughputException, CustomerApplicationException { + // Create new lease for the child shards if they don't exist. + // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. + // This would happen when KinesisDataFetcher catches ResourceNotFound exception. + // In this case, KinesisDataFetcher will send out SHARD_END signal to trigger a shutdown task with empty list of childShards. + // This scenario could happen when customer deletes the stream while leaving the KCL application running. + if (currentShardLease == null) { + throw new InvalidStateException("Shard " + shardInfo.getShardId() + ": Lease not owned by the current worker. Leaving ShardEnd handling to new owner."); + } + if (!CollectionUtils.isNullOrEmpty(childShards)) { + // If childShards is not empty, create new leases for the childShards and update the current lease with the childShards lease information. + createLeasesForChildShardsIfNotExist(); + updateCurrentLeaseWithChildShards(currentShardLease); + } else { + LOG.warn("Shard " + shardInfo.getShardId() + + ": Shutting down consumer with SHARD_END reason without creating leases for child shards."); + } + // Checkpoint with SHARD_END sequence number. + final LeasePendingDeletion leasePendingDeletion = new LeasePendingDeletion(currentShardLease, shardInfo); + if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { + boolean isSuccess = false; + try { + isSuccess = attemptShardEndCheckpointing(startTime); + } finally { + // Check if either the shard end ddb persist is successful or + // if childshards is empty. When child shards is empty then either it is due to + // completed shard being reprocessed or we got RNF from service. + // For these cases enqueue the lease for deletion. + if (isSuccess || CollectionUtils.isNullOrEmpty(childShards)) { + leaseCleanupManager.enqueueForDeletion(leasePendingDeletion); + } + } + } + } + + private void takeLeaseLostAction() { + final ShutdownInput leaseLostShutdownInput = new ShutdownInput() + .withShutdownReason(ShutdownReason.ZOMBIE) + .withCheckpointer(recordProcessorCheckpointer); + recordProcessor.shutdown(leaseLostShutdownInput); + } + + private boolean attemptShardEndCheckpointing(long startTime) + throws DependencyException, ProvisionedThroughputException, InvalidStateException, CustomerApplicationException { + final KinesisClientLease leaseFromDdb = Optional.ofNullable(leaseCoordinator.getLeaseManager().getLease(shardInfo.getShardId())) + .orElseThrow(() -> new InvalidStateException("Lease for shard " + shardInfo.getShardId() + " does not exist.")); + if (!leaseFromDdb.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END)) { + // Call the recordProcessor to checkpoint with SHARD_END sequence number. + // The recordProcessor.shutdown is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling recordProcessor.shutdown. + throwOnApplicationException(() -> applicationCheckpointAndVerification(), startTime); + } + return true; + } + + private void applicationCheckpointAndVerification() { + recordProcessorCheckpointer.setSequenceNumberAtShardEnd( + recordProcessorCheckpointer.getLargestPermittedCheckpointValue()); + recordProcessorCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); + final ShutdownInput shardEndShutdownInput = new ShutdownInput() + .withShutdownReason(ShutdownReason.TERMINATE) + .withCheckpointer(recordProcessorCheckpointer); + recordProcessor.shutdown(shardEndShutdownInput); + + final ExtendedSequenceNumber lastCheckpointValue = recordProcessorCheckpointer.getLastCheckpointValue(); + + final boolean successfullyCheckpointedShardEnd = lastCheckpointValue.equals(ExtendedSequenceNumber.SHARD_END); + + if ((lastCheckpointValue == null) || (!successfullyCheckpointedShardEnd)) { + throw new IllegalArgumentException("Application didn't checkpoint at end of shard " + + shardInfo.getShardId() + ". Application must checkpoint upon shutdown. " + + "See IRecordProcessor.shutdown javadocs for more information."); + } + } + + private void throwOnApplicationException(Runnable action, long startTime) throws CustomerApplicationException { + try { + action.run(); + } catch (Exception e) { + throw new CustomerApplicationException("Customer application throws exception for shard " + shardInfo.getShardId(), e); + } finally { + MetricsHelper.addLatency(RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); } - return shutdownReason; } private void createLeasesForChildShardsIfNotExist() throws InvalidStateException, DependencyException, ProvisionedThroughputException { @@ -285,13 +324,12 @@ class ShutdownTask implements ITask { return reason; } - private void dropLease() { - KinesisClientLease lease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); - if (lease == null) { - LOG.warn("Shard " + shardInfo.getShardId() + ": Lease already dropped. Will shutdown the shardConsumer directly."); + private void dropLease(KinesisClientLease currentShardLease) { + if (currentShardLease == null) { + LOG.warn("Shard " + shardInfo.getShardId() + ": Unable to find the lease for shard. Will shutdown the shardConsumer directly."); return; } - leaseCoordinator.dropLease(lease); - LOG.warn("Dropped lease for shutting down ShardConsumer: " + lease.getLeaseKey()); + leaseCoordinator.dropLease(currentShardLease); + LOG.warn("Dropped lease for shutting down ShardConsumer: " + currentShardLease.getLeaseKey()); } } diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/exceptions/CustomerApplicationException.java b/src/main/java/com/amazonaws/services/kinesis/leases/exceptions/CustomerApplicationException.java new file mode 100644 index 00000000..1ef906af --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/leases/exceptions/CustomerApplicationException.java @@ -0,0 +1,24 @@ +/* + * Copyright 2019 Amazon.com, Inc. or its affiliates. + * Licensed under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.amazonaws.services.kinesis.leases.exceptions; + +public class CustomerApplicationException extends Exception { + public CustomerApplicationException(Throwable t) {super(t);} + + public CustomerApplicationException(String message, Throwable t) {super(message, t);} + + public CustomerApplicationException(String message) {super(message);} +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 67bf3697..cb89b619 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -583,7 +583,12 @@ public class ShardConsumerTest { .thenReturn(getRecordsCache); when(leaseManager.getLease(anyString())).thenReturn(null); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(new KinesisClientLease()); + List parentShardIds = new ArrayList<>(); + parentShardIds.add("parentShardId"); + KinesisClientLease currentLease = createLease(streamShardId, "leaseOwner", parentShardIds); + currentLease.setCheckpoint(new ExtendedSequenceNumber("testSequenceNumbeer")); + when(leaseManager.getLease(streamShardId)).thenReturn(currentLease); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(currentLease); RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( shardInfo, @@ -705,7 +710,11 @@ public class ShardConsumerTest { final int idleTimeMS = 0; // keep unit tests fast ICheckpoint checkpoint = new InMemoryCheckpointImpl(startSeqNum.toString()); checkpoint.setCheckpoint(streamShardId, ExtendedSequenceNumber.TRIM_HORIZON, testConcurrencyToken); - when(leaseManager.getLease(anyString())).thenReturn(null); + List parentShardIds = new ArrayList<>(); + parentShardIds.add("parentShardId"); + KinesisClientLease currentLease = createLease(streamShardId, "leaseOwner", parentShardIds); + currentLease.setCheckpoint(new ExtendedSequenceNumber("testSequenceNumbeer")); + when(leaseManager.getLease(streamShardId)).thenReturn(currentLease); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); TransientShutdownErrorTestStreamlet processor = new TransientShutdownErrorTestStreamlet(); @@ -758,11 +767,7 @@ public class ShardConsumerTest { shardSyncer, shardSyncStrategy); - List parentShardIds = new ArrayList<>(); - parentShardIds.add(shardInfo.getShardId()); - when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(createLease(shardInfo.getShardId(), - "leaseOwner", - parentShardIds)); + when(leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId())).thenReturn(currentLease); when(leaseCoordinator.updateLease(any(KinesisClientLease.class), any(UUID.class))).thenReturn(true); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 219fd28c..2942dcb7 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -38,9 +38,12 @@ import java.util.UUID; import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.BlockedOnParentShardException; import com.amazonaws.services.kinesis.clientlibrary.proxies.ShardListWrappingShardClosureVerificationResponse; import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; +import com.amazonaws.services.kinesis.leases.exceptions.CustomerApplicationException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.impl.UpdateField; import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; @@ -81,6 +84,7 @@ public class ShutdownTaskTest { defaultParentShardIds, ExtendedSequenceNumber.LATEST); ShardSyncer shardSyncer = new KinesisShardSyncer(new KinesisLeaseCleanupValidator()); + IMetricsFactory metricsFactory = new NullMetricsFactory(); @Mock @@ -119,8 +123,11 @@ public class ShutdownTaskTest { public void setUp() throws Exception { doNothing().when(getRecordsCache).shutdown(); final KinesisClientLease parentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); + parentLease.setCheckpoint(new ExtendedSequenceNumber("3298")); when(leaseCoordinator.getLeaseManager()).thenReturn(leaseManager); when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn(parentLease); + when(leaseManager.getLease(defaultShardId)).thenReturn(parentLease); + } /** @@ -154,12 +161,12 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructSplitChildShards(), - leaseCleanupManager); + leaseCleanupManager, + metricsFactory); TaskResult result = task.call(); assertNotNull(result.getException()); - Assert.assertTrue(result.getException() instanceof IllegalArgumentException); - final String expectedExceptionMessage = "Application didn't checkpoint at end of shard shardId-0. " + - "Application must checkpoint upon shutdown. See IRecordProcessor.shutdown javadocs for more information."; + Assert.assertTrue(result.getException() instanceof CustomerApplicationException); + final String expectedExceptionMessage = "Customer application throws exception for shard shardId-0"; Assert.assertEquals(expectedExceptionMessage, result.getException().getMessage()); } @@ -190,7 +197,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructSplitChildShards(), - leaseCleanupManager); + leaseCleanupManager, + metricsFactory); TaskResult result = task.call(); verify(getRecordsCache).shutdown(); verify(leaseCoordinator).dropLease(any(KinesisClientLease.class)); @@ -206,6 +214,7 @@ public class ShutdownTaskTest { boolean ignoreUnexpectedChildShards = false; KinesisClientLease currentLease = createLease(defaultShardId, "leaseOwner", Collections.emptyList()); + currentLease.setCheckpoint(new ExtendedSequenceNumber("3298")); KinesisClientLease adjacentParentLease = createLease("ShardId-1", "leaseOwner", Collections.emptyList()); when(leaseCoordinator.getCurrentlyHeldLease(defaultShardId)).thenReturn( currentLease); when(leaseManager.getLease(defaultShardId)).thenReturn(currentLease); @@ -227,7 +236,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager)); + leaseCleanupManager, + metricsFactory)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); @@ -253,7 +263,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager)); + leaseCleanupManager, + metricsFactory)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNull(result.getException()); @@ -291,7 +302,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager)); + leaseCleanupManager, + metricsFactory)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); @@ -316,7 +328,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager)); + leaseCleanupManager, + metricsFactory)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(true); TaskResult result = task.call(); assertNull(result.getException()); @@ -347,7 +360,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructSplitChildShards(), - leaseCleanupManager); + leaseCleanupManager, + metricsFactory); TaskResult result = task.call(); verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseManager).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); @@ -381,7 +395,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, Collections.emptyList(), - leaseCleanupManager); + leaseCleanupManager, + metricsFactory); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); @@ -411,7 +426,8 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, Collections.emptyList(), - leaseCleanupManager); + leaseCleanupManager, + metricsFactory); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); @@ -428,7 +444,8 @@ public class ShutdownTaskTest { ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, leaseCoordinator, 0, - getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList(), leaseCleanupManager); + getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList(), + leaseCleanupManager, metricsFactory); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } From f7130175282b62627bb6538f728dc797ae215a64 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 28 Jul 2020 13:11:57 -0700 Subject: [PATCH 19/43] LeaseCleanupManager change --- .../clientlibrary/lib/worker/Worker.java | 7 +------ .../leases/impl/LeaseCleanupManager.java | 17 ++++++++++------- .../leases/impl/LeaseCleanupManagerTest.java | 11 +++++++++++ 3 files changed, 22 insertions(+), 13 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 5b4f31e8..8e5cce85 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -733,12 +733,7 @@ public class Worker implements Runnable { } } - 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 the // shard sync in the previous attempt. diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java index af02f588..b3157e78 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManager.java @@ -114,13 +114,16 @@ public class LeaseCleanupManager { * {@link LeaseCleanupManager#leaseCleanupIntervalMillis} */ public void start() { - LOG.debug("Starting lease cleanup thread."); - isRunning = true; - completedLeaseStopwatch.start(); - garbageLeaseStopwatch.start(); - - deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, - TimeUnit.MILLISECONDS); + if (!isRunning) { + LOG.info("Starting lease cleanup thread."); + completedLeaseStopwatch.start(); + garbageLeaseStopwatch.start(); + deletionThreadPool.scheduleAtFixedRate(new LeaseCleanupThread(), INITIAL_DELAY, leaseCleanupIntervalMillis, + TimeUnit.MILLISECONDS); + isRunning = true; + } else { + LOG.info("Lease cleanup thread already running, no need to start."); + } } /** diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java index 367c0ab0..f89ae644 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseCleanupManagerTest.java @@ -26,6 +26,7 @@ import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -82,6 +83,16 @@ public class LeaseCleanupManagerTest { cleanupLeasesOfCompletedShards, leaseCleanupIntervalMillis, completedLeaseCleanupIntervalMillis, garbageLeaseCleanupIntervalMillis, maxRecords); } + /** + * 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 6738087a8fc5fef7b7fdb7a3cbf32bfbbcccdd4a Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 28 Jul 2020 13:32:04 -0700 Subject: [PATCH 20/43] updateLeaseWithMetaInfo with expectation --- .../leases/impl/KinesisClientLeaseSerializer.java | 5 +++++ .../services/kinesis/leases/impl/LeaseManager.java | 2 +- .../services/kinesis/leases/impl/LeaseSerializer.java | 10 ++++++++++ .../kinesis/leases/interfaces/ILeaseSerializer.java | 7 +++++++ 4 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 0b9271be..966faee4 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -120,6 +120,11 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer getDynamoExistantExpectation(final String leaseKey) { + return baseSerializer.getDynamoExistantExpectation(leaseKey); + } + @Override public Map getDynamoLeaseCounterUpdate(KinesisClientLease lease) { return baseSerializer.getDynamoLeaseCounterUpdate(lease); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index e5860870..36f56dda 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -620,7 +620,7 @@ public class LeaseManager implements ILeaseManager { UpdateItemRequest request = new UpdateItemRequest(); request.setTableName(table); request.setKey(serializer.getDynamoHashKey(lease)); - request.setExpected(serializer.getDynamoLeaseCounterExpectation(lease)); + request.setExpected(serializer.getDynamoExistantExpectation(lease.getLeaseKey())); Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); updates.putAll(serializer.getDynamoUpdateLeaseUpdate(lease)); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java index b02ed34c..62977df7 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java @@ -137,6 +137,16 @@ public class LeaseSerializer implements ILeaseSerializer { return result; } + @Override + public Map getDynamoExistantExpectation(final String leaseKey) { + Map result = new HashMap<>(); + + ExpectedAttributeValue expectedAV = new ExpectedAttributeValue(DynamoUtils.createAttributeValue(leaseKey)); + result.put(LEASE_KEY_KEY, expectedAV); + + return result; + } + @Override public Map getDynamoLeaseCounterUpdate(Lease lease) { return getDynamoLeaseCounterUpdate(lease.getLeaseCounter()); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java index 58eb6613..8ff56b7d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java @@ -79,6 +79,13 @@ public interface ILeaseSerializer { */ public Map getDynamoNonexistantExpectation(); + /** + * @return the attribute value map asserting that a lease does exist. + */ + default Map getDynamoExistantExpectation(final String leaseKey) { + throw new UnsupportedOperationException("DynamoExistantExpectation is not implemented"); + } + /** * @param lease * @return the attribute value map that increments a lease counter From 38cef8963a442aab43f8bb0356e3e5d99e2f1279 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 28 Jul 2020 13:43:56 -0700 Subject: [PATCH 21/43] fixing expectation --- .../services/kinesis/leases/impl/LeaseSerializer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java index 62977df7..29bf0f9b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java @@ -141,7 +141,9 @@ public class LeaseSerializer implements ILeaseSerializer { public Map getDynamoExistantExpectation(final String leaseKey) { Map result = new HashMap<>(); - ExpectedAttributeValue expectedAV = new ExpectedAttributeValue(DynamoUtils.createAttributeValue(leaseKey)); + ExpectedAttributeValue expectedAV = new ExpectedAttributeValue(); + expectedAV.setValue(DynamoUtils.createAttributeValue(leaseKey)); + expectedAV.setExists(true); result.put(LEASE_KEY_KEY, expectedAV); return result; From 22737c4a5be03754da076d1708d92295745f6d1c Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Tue, 28 Jul 2020 17:51:50 -0700 Subject: [PATCH 22/43] addressing comments --- .../lib/worker/ConsumerStates.java | 3 +- .../lib/worker/ShardConsumer.java | 1 - .../lib/worker/ShutdownTask.java | 99 ++++++++----------- .../impl/KinesisClientLeaseSerializer.java | 4 +- .../kinesis/leases/impl/LeaseManager.java | 4 +- .../kinesis/leases/impl/LeaseSerializer.java | 2 +- .../leases/interfaces/ILeaseSerializer.java | 4 +- .../lib/worker/ShutdownTaskTest.java | 30 ++---- .../impl/LeaseManagerIntegrationTest.java | 32 ++++++ 9 files changed, 93 insertions(+), 86 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index b5cde2bc..5cf55dbf 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -532,8 +532,7 @@ class ConsumerStates { consumer.getTaskBackoffTimeMillis(), consumer.getGetRecordsCache(), consumer.getShardSyncer(), consumer.getShardSyncStrategy(), consumer.getChildShards(), - consumer.getLeaseCleanupManager(), - consumer.getMetricsFactory()); + consumer.getLeaseCleanupManager()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 10600def..29a95ac4 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -54,7 +54,6 @@ class ShardConsumer { private final ExecutorService executorService; private final ShardInfo shardInfo; private final KinesisDataFetcher dataFetcher; - @Getter private final IMetricsFactory metricsFactory; private final KinesisClientLibLeaseCoordinator leaseCoordinator; private ICheckpoint checkpoint; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index a9ec8d76..274aaaa1 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -22,7 +22,6 @@ import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; import com.amazonaws.services.kinesis.leases.exceptions.ProvisionedThroughputException; import com.amazonaws.services.kinesis.leases.impl.LeaseCleanupManager; import com.amazonaws.services.kinesis.leases.impl.UpdateField; -import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.ChildShard; import com.amazonaws.util.CollectionUtils; import org.apache.commons.logging.Log; @@ -33,8 +32,6 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; -import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; -import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.google.common.annotations.VisibleForTesting; import java.util.List; @@ -51,10 +48,8 @@ class ShutdownTask implements ITask { private static final Log LOG = LogFactory.getLog(ShutdownTask.class); - 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 = 50; private final ShardInfo shardInfo; private final IRecordProcessor recordProcessor; @@ -72,7 +67,6 @@ class ShutdownTask implements ITask { private final ShardSyncStrategy shardSyncStrategy; private final List childShards; private final LeaseCleanupManager leaseCleanupManager; - private final IMetricsFactory metricsFactory; /** * Constructor. @@ -90,7 +84,7 @@ class ShutdownTask implements ITask { long backoffTimeMillis, GetRecordsCache getRecordsCache, ShardSyncer shardSyncer, ShardSyncStrategy shardSyncStrategy, List childShards, - LeaseCleanupManager leaseCleanupManager, IMetricsFactory metricsFactory) { + LeaseCleanupManager leaseCleanupManager) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -106,7 +100,6 @@ class ShutdownTask implements ITask { this.shardSyncStrategy = shardSyncStrategy; this.childShards = childShards; this.leaseCleanupManager = leaseCleanupManager; - this.metricsFactory = metricsFactory; } /* @@ -117,61 +110,55 @@ class ShutdownTask implements ITask { */ @Override public TaskResult call() { - MetricsHelper.startScope(metricsFactory, SHUTDOWN_TASK_OPERATION); Exception exception; + LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: " + + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); + try { - LOG.info("Invoking shutdown() for shard " + shardInfo.getShardId() + ", concurrencyToken: " - + shardInfo.getConcurrencyToken() + ", original Shutdown reason: " + reason + ". childShards:" + childShards); + final KinesisClientLease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); + final Runnable leaseLostAction = () -> takeLeaseLostAction(); - try { - final long startTime = System.currentTimeMillis(); - final KinesisClientLease currentShardLease = leaseCoordinator.getCurrentlyHeldLease(shardInfo.getShardId()); - final Runnable leaseLostAction = () -> takeLeaseLostAction(); - - if (reason == ShutdownReason.TERMINATE) { - try { - takeShardEndAction(currentShardLease, 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 ZOMBIE reason to allow other worker to take the lease and retry shutting down. - LOG.warn("Lease " + shardInfo.getShardId() + ": Invalid state encountered while shutting down shardConsumer with TERMINATE reason. " + - "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. ", e); - dropLease(currentShardLease); - throwOnApplicationException(leaseLostAction, startTime); - } - } else { - throwOnApplicationException(leaseLostAction, startTime); - } - - LOG.debug("Shutting down retrieval strategy."); - getRecordsCache.shutdown(); - LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId()); - return new TaskResult(null); - } catch (Exception e) { - if (e instanceof CustomerApplicationException) { - LOG.error("Shard " + shardInfo.getShardId() + ": Application exception: ", e); - } else { - LOG.error("Shard " + shardInfo.getShardId() + ": Caught exception: ", e); - } - - exception = e; - // backoff if we encounter an exception. + if (reason == ShutdownReason.TERMINATE) { try { - Thread.sleep(this.backoffTimeMillis); - } catch (InterruptedException ie) { - LOG.debug("Interrupted sleep", ie); + takeShardEndAction(currentShardLease); + } catch (InvalidStateException e) { + // If InvalidStateException happens, it indicates we have a non recoverable error in short term. + // In this scenario, we should shutdown the shardConsumer with ZOMBIE reason to allow other worker to take the lease and retry shutting down. + LOG.warn("Lease " + shardInfo.getShardId() + ": Invalid state encountered while shutting down shardConsumer with TERMINATE reason. " + + "Dropping the lease and shutting down shardConsumer using ZOMBIE reason. ", e); + dropLease(currentShardLease); + throwOnApplicationException(leaseLostAction); } + } else { + throwOnApplicationException(leaseLostAction); + } + + LOG.debug("Shutting down retrieval strategy."); + getRecordsCache.shutdown(); + LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId()); + return new TaskResult(null); + } catch (Exception e) { + if (e instanceof CustomerApplicationException) { + LOG.error("Shard " + shardInfo.getShardId() + ": Application exception: ", e); + } else { + LOG.error("Shard " + shardInfo.getShardId() + ": Caught exception: ", e); + } + + exception = e; + // backoff if we encounter an exception. + try { + Thread.sleep(this.backoffTimeMillis); + } catch (InterruptedException ie) { + LOG.debug("Interrupted sleep", ie); } - } finally { - MetricsHelper.endScope(); } return new TaskResult(exception); } // Involves persisting child shard info, attempt to checkpoint and enqueueing lease for cleanup. - private void takeShardEndAction(KinesisClientLease currentShardLease, long startTime) + private void takeShardEndAction(KinesisClientLease currentShardLease) throws InvalidStateException, DependencyException, ProvisionedThroughputException, CustomerApplicationException { // Create new lease for the child shards if they don't exist. // We have one valid scenario that shutdown task got created with SHARD_END reason and an empty list of childShards. @@ -194,7 +181,7 @@ class ShutdownTask implements ITask { if (!leaseCleanupManager.isEnqueuedForDeletion(leasePendingDeletion)) { boolean isSuccess = false; try { - isSuccess = attemptShardEndCheckpointing(startTime); + isSuccess = attemptShardEndCheckpointing(); } finally { // Check if either the shard end ddb persist is successful or // if childshards is empty. When child shards is empty then either it is due to @@ -214,14 +201,14 @@ class ShutdownTask implements ITask { recordProcessor.shutdown(leaseLostShutdownInput); } - private boolean attemptShardEndCheckpointing(long startTime) + private boolean attemptShardEndCheckpointing() throws DependencyException, ProvisionedThroughputException, InvalidStateException, CustomerApplicationException { final KinesisClientLease leaseFromDdb = Optional.ofNullable(leaseCoordinator.getLeaseManager().getLease(shardInfo.getShardId())) .orElseThrow(() -> new InvalidStateException("Lease for shard " + shardInfo.getShardId() + " does not exist.")); if (!leaseFromDdb.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END)) { // Call the recordProcessor to checkpoint with SHARD_END sequence number. // The recordProcessor.shutdown is implemented by customer. We should validate if the SHARD_END checkpointing is successful after calling recordProcessor.shutdown. - throwOnApplicationException(() -> applicationCheckpointAndVerification(), startTime); + throwOnApplicationException(() -> applicationCheckpointAndVerification()); } return true; } @@ -246,13 +233,11 @@ class ShutdownTask implements ITask { } } - private void throwOnApplicationException(Runnable action, long startTime) throws CustomerApplicationException { + private void throwOnApplicationException(Runnable action) throws CustomerApplicationException { try { action.run(); } catch (Exception e) { throw new CustomerApplicationException("Customer application throws exception for shard " + shardInfo.getShardId(), e); - } finally { - MetricsHelper.addLatency(RECORD_PROCESSOR_SHUTDOWN_METRIC, startTime, MetricsLevel.SUMMARY); } } diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 966faee4..310edb67 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -121,8 +121,8 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer getDynamoExistantExpectation(final String leaseKey) { - return baseSerializer.getDynamoExistantExpectation(leaseKey); + public Map getDynamoExistentExpectation(final String leaseKey) { + return baseSerializer.getDynamoExistentExpectation(leaseKey); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index 36f56dda..0c70aaa7 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -620,7 +620,7 @@ public class LeaseManager implements ILeaseManager { UpdateItemRequest request = new UpdateItemRequest(); request.setTableName(table); request.setKey(serializer.getDynamoHashKey(lease)); - request.setExpected(serializer.getDynamoExistantExpectation(lease.getLeaseKey())); + request.setExpected(serializer.getDynamoExistentExpectation(lease.getLeaseKey())); Map updates = serializer.getDynamoUpdateLeaseUpdate(lease, updateField); updates.putAll(serializer.getDynamoUpdateLeaseUpdate(lease)); @@ -628,6 +628,8 @@ public class LeaseManager implements ILeaseManager { try { dynamoDBClient.updateItem(request); + } catch (ConditionalCheckFailedException e) { + LOG.warn("Lease update failed for lease with key " + lease.getLeaseKey() + " because the lease did not exist at the time of the update", e); } catch (AmazonClientException e) { throw convertAndRethrowExceptions("update", lease.getLeaseKey(), e); } diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java index 29bf0f9b..85381560 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseSerializer.java @@ -138,7 +138,7 @@ public class LeaseSerializer implements ILeaseSerializer { } @Override - public Map getDynamoExistantExpectation(final String leaseKey) { + public Map getDynamoExistentExpectation(final String leaseKey) { Map result = new HashMap<>(); ExpectedAttributeValue expectedAV = new ExpectedAttributeValue(); diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java index 8ff56b7d..2d9ea0c9 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/interfaces/ILeaseSerializer.java @@ -82,8 +82,8 @@ public interface ILeaseSerializer { /** * @return the attribute value map asserting that a lease does exist. */ - default Map getDynamoExistantExpectation(final String leaseKey) { - throw new UnsupportedOperationException("DynamoExistantExpectation is not implemented"); + default Map getDynamoExistentExpectation(final String leaseKey) { + throw new UnsupportedOperationException("DynamoExistentExpectation is not implemented"); } /** diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 2942dcb7..8b67f5dc 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -161,8 +161,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructSplitChildShards(), - leaseCleanupManager, - metricsFactory); + leaseCleanupManager); TaskResult result = task.call(); assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof CustomerApplicationException); @@ -197,8 +196,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructSplitChildShards(), - leaseCleanupManager, - metricsFactory); + leaseCleanupManager); TaskResult result = task.call(); verify(getRecordsCache).shutdown(); verify(leaseCoordinator).dropLease(any(KinesisClientLease.class)); @@ -236,8 +234,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager, - metricsFactory)); + leaseCleanupManager)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); @@ -263,8 +260,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager, - metricsFactory)); + leaseCleanupManager)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNull(result.getException()); @@ -302,8 +298,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager, - metricsFactory)); + leaseCleanupManager)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(false); TaskResult result = task.call(); assertNotNull(result.getException()); @@ -328,8 +323,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructMergeChildShards(), - leaseCleanupManager, - metricsFactory)); + leaseCleanupManager)); when(task.isOneInNProbability(RETRY_RANDOM_MAX_RANGE)).thenReturn(true); TaskResult result = task.call(); assertNull(result.getException()); @@ -360,8 +354,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, constructSplitChildShards(), - leaseCleanupManager, - metricsFactory); + leaseCleanupManager); TaskResult result = task.call(); verify(leaseManager, times(2)).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseManager).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); @@ -395,8 +388,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, Collections.emptyList(), - leaseCleanupManager, - metricsFactory); + leaseCleanupManager); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); @@ -426,8 +418,7 @@ public class ShutdownTaskTest { shardSyncer, shardSyncStrategy, Collections.emptyList(), - leaseCleanupManager, - metricsFactory); + leaseCleanupManager); TaskResult result = task.call(); verify(leaseManager, never()).createLeaseIfNotExists(any(KinesisClientLease.class)); verify(leaseManager, never()).updateLeaseWithMetaInfo(any(KinesisClientLease.class), any(UpdateField.class)); @@ -444,8 +435,7 @@ public class ShutdownTaskTest { ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, leaseCoordinator, 0, - getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList(), - leaseCleanupManager, metricsFactory); + getRecordsCache, shardSyncer, shardSyncStrategy, Collections.emptyList(), leaseCleanupManager); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java index d0b0813d..c04ee44f 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java @@ -27,6 +27,7 @@ import com.amazonaws.services.dynamodbv2.model.ListTablesResult; import com.amazonaws.services.dynamodbv2.model.TableDescription; import com.amazonaws.services.dynamodbv2.model.TableStatus; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration; +import com.amazonaws.services.kinesis.model.HashKeyRange; import junit.framework.Assert; import org.junit.Test; @@ -124,6 +125,37 @@ public class LeaseManagerIntegrationTest extends LeaseIntegrationTest { Assert.assertFalse(leaseManager.renewLease(leaseCopy)); } + /** + * Tests leaseManager.updateLeaseWithMetaInfo() when the lease is deleted before updating it with meta info + */ + @Test + public void testDeleteLeaseThenUpdateLeaseWithMetaInfo() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseManager); + KinesisClientLease lease = builder.withLease("1").build().get("1"); + final String leaseKey = lease.getLeaseKey(); + leaseManager.deleteLease(lease); + leaseManager.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + final KinesisClientLease deletedLease = leaseManager.getLease(leaseKey); + Assert.assertNull(deletedLease); + } + + /** + * Tests leaseManager.updateLeaseWithMetaInfo() on hashKeyRange update + */ + @Test + public void testUpdateLeaseWithMetaInfo() throws LeasingException { + TestHarnessBuilder builder = new TestHarnessBuilder(leaseManager); + KinesisClientLease lease = builder.withLease("1").build().get("1"); + final String leaseKey = lease.getLeaseKey(); + final HashKeyRangeForLease hashKeyRangeForLease = HashKeyRangeForLease.fromHashKeyRange(new HashKeyRange() + .withStartingHashKey("1") + .withEndingHashKey("2")); + lease.setHashKeyRange(hashKeyRangeForLease); + leaseManager.updateLeaseWithMetaInfo(lease, UpdateField.HASH_KEY_RANGE); + final KinesisClientLease updatedLease = leaseManager.getLease(leaseKey); + Assert.assertEquals(lease, updatedLease); + } + /** * Tests takeLease when the lease is not already owned. */ From 0d358b682da90f8d002520d47aff8534d0b36382 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Fri, 31 Jul 2020 16:29:25 -0700 Subject: [PATCH 23/43] metric name change for periodicShardSyncManager --- .../clientlibrary/lib/worker/PeriodicShardSyncManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java index 1f3b9cff..cdf73e82 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PeriodicShardSyncManager.java @@ -181,8 +181,8 @@ class PeriodicShardSyncManager { try { final ShardSyncResponse shardSyncResponse = checkForShardSync(); - MetricsHelper.getMetricsScope().addData("InitiatingShardSync", shardSyncResponse.shouldDoShardSync() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY); - MetricsHelper.getMetricsScope().addData("DetectedIncompleteLease", shardSyncResponse.isHoleDetected() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY); + MetricsHelper.getMetricsScope().addData("NumStreamsToSync", shardSyncResponse.shouldDoShardSync() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY); + MetricsHelper.getMetricsScope().addData("NumStreamsWithPartialLeases", shardSyncResponse.isHoleDetected() ? 1 : 0, StandardUnit.Count, MetricsLevel.SUMMARY); if (shardSyncResponse.shouldDoShardSync()) { LOG.info("Periodic shard syncer initiating shard sync due to the reason - " + shardSyncResponse.reasonForDecision()); From 17ecc87d71e0f4f9f9264a0f20c6a5f18b2a5219 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Tue, 9 Jun 2020 05:57:24 -0400 Subject: [PATCH 24/43] temp pom change --- pom.xml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 0f559b7d..31f67e35 100644 --- a/pom.xml +++ b/pom.xml @@ -37,10 +37,15 @@ aws-java-sdk-dynamodb ${aws-java-sdk.version} + + + + + - com.amazonaws - aws-java-sdk-kinesis - ${aws-java-sdk.version} + dummy + test + 1.0 com.amazonaws From 41c08c3b04487cd44efb4909c644034bd09c7fd9 Mon Sep 17 00:00:00 2001 From: Cai41 Date: Wed, 22 Apr 2020 18:11:51 -0700 Subject: [PATCH 25/43] Also consider TableStatus.UPDATING status --- .../kinesis/leases/impl/LeaseManager.java | 3 ++- .../impl/LeaseManagerIntegrationTest.java | 23 +++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index 0c70aaa7..00a3e755 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -162,7 +162,8 @@ public class LeaseManager implements ILeaseManager { */ @Override public boolean leaseTableExists() throws DependencyException { - return TableStatus.ACTIVE == tableStatus(); + TableStatus tableStatus = tableStatus(); + return TableStatus.ACTIVE == tableStatus || TableStatus.UPDATING == tableStatus; } private TableStatus tableStatus() throws DependencyException { diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java index c04ee44f..6174cdf0 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java @@ -296,6 +296,29 @@ public class LeaseManagerIntegrationTest extends LeaseIntegrationTest { Assert.assertTrue(manager.waitUntilLeaseTableExists(1, 1)); } + @Test + public void testWaitUntilLeaseTableExistsUpdatingStatus() throws LeasingException { + AmazonDynamoDBClient ddbMock = Mockito.mock(ddbClient.getClass()); + DescribeTableResult result = Mockito.mock(DescribeTableResult.class); + TableDescription description = Mockito.mock(TableDescription.class); + Mockito.when(description.getTableStatus()).thenReturn(TableStatus.UPDATING.name()); + Mockito.when(result.getTable()).thenReturn(description); + Mockito.when(ddbMock.describeTable(Mockito.any(DescribeTableRequest.class))).thenReturn(result); + KinesisClientLeaseManager manager = new KinesisClientLeaseManager("existing_table", ddbMock, true, + KinesisClientLibConfiguration.DEFAULT_DDB_BILLING_MODE) { + + @Override + long sleep(long timeToSleepMillis) { + Assert.fail("Should not sleep"); + return 0L; + } + + }; + + + Assert.assertTrue(manager.waitUntilLeaseTableExists(1, 1)); + } + @Test public void testWaitUntilLeaseTableExistsPayPerRequest() throws LeasingException { AmazonDynamoDBClient ddbMock = Mockito.mock(ddbClient.getClass()); From 44474a1c0574eb4184b9d709f7c1981609d99f04 Mon Sep 17 00:00:00 2001 From: Keerthy Muralidharan Date: Thu, 14 May 2020 13:45:54 -0700 Subject: [PATCH 26/43] protobuf upgrade --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 31f67e35..f80bc399 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,7 @@ com.google.protobuf protobuf-java - 2.6.1 + 3.11.4 org.apache.commons From bb84c17abffb6426f35d2314be312e1157a0333a Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Thu, 6 Aug 2020 10:03:03 -0700 Subject: [PATCH 27/43] Lease cleanup interval fix --- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 3f06ea40..90389a44 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -93,7 +93,7 @@ public class KinesisClientLibConfiguration { /** * Interval to run lease cleanup thread in {@link LeaseCleanupManager}. */ - private static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofHours(1).toMillis(); + private static final long DEFAULT_LEASE_CLEANUP_INTERVAL_MILLIS = Duration.ofMinutes(1).toMillis(); /** * Threshold in millis at which to check if there are any completed leases (leases for shards which have been From 66bcd45d366fc6ea5a164dfb570052694af77335 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 7 Aug 2020 14:35:13 -0400 Subject: [PATCH 28/43] Using correct default shard prioritization strategy. * Fixing bug where this was set as default in configuration, but if a configuraiton is not specified it falls back to parent prioritization. --- .../services/kinesis/clientlibrary/lib/worker/Worker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 38d616c9..f8c66181 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -1469,7 +1469,7 @@ public class Worker implements Runnable { } if (shardPrioritization == null) { - shardPrioritization = new ParentsFirstShardPrioritization(1); + shardPrioritization = new NoOpShardPrioritization(); } if (kinesisProxy == null) { From 1f71042d8022b19cc1f9b991b9f8cc5bd53bf370 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 7 Aug 2020 14:43:38 -0400 Subject: [PATCH 29/43] Revert "temp pom change" This reverts commit 129eee72da9be4f224f0ec23f08f305fb9564cde. --- pom.xml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/pom.xml b/pom.xml index f80bc399..35f8f05c 100644 --- a/pom.xml +++ b/pom.xml @@ -37,15 +37,10 @@ aws-java-sdk-dynamodb ${aws-java-sdk.version} - - - - - - dummy - test - 1.0 + com.amazonaws + aws-java-sdk-kinesis + ${aws-java-sdk.version} com.amazonaws From 34e9016c01fc8e8e1b8181baa23ec865a359be00 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 7 Aug 2020 14:45:25 -0400 Subject: [PATCH 30/43] Deprecating class --- .../lib/worker/ParentsFirstShardPrioritization.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java index 6359b3b1..126eb6ea 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ParentsFirstShardPrioritization.java @@ -25,6 +25,7 @@ import java.util.Map; * It also limits number of shards that will be available for initialization based on their depth. * It doesn't make a lot of sense to work on a shard that has too many unfinished parents. */ +@Deprecated public class ParentsFirstShardPrioritization implements ShardPrioritization { private static final SortingNode PROCESSING_NODE = new SortingNode(null, Integer.MIN_VALUE); From a85ef70898984fe8dfeaeffa6ec7f8d82fdae36d Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 7 Aug 2020 15:07:10 -0400 Subject: [PATCH 31/43] Fixing list shards bug --- .../kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java index 1ce1175a..4235bad9 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisShardSyncer.java @@ -169,7 +169,7 @@ class KinesisShardSyncer implements ShardSyncer { List shards; if(CollectionUtils.isNullOrEmpty(latestShards)) { - shards = getShardListAtInitialPosition(kinesisProxy, initialPosition); + shards = isLeaseTableEmpty ? getShardListAtInitialPosition(kinesisProxy, initialPosition) : getCompleteShardList(kinesisProxy); } else { shards = latestShards; } From 700daa7a87bd614c01a3f7b3e92d0912173ef876 Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Fri, 7 Aug 2020 19:34:27 -0400 Subject: [PATCH 32/43] Fixing cache miss bug in list shards --- .../clientlibrary/lib/worker/ShardSyncerTest.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index e2cc578c..9e9870d3 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -70,6 +70,7 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -521,8 +522,6 @@ public class ShardSyncerTest { for (int c = 1; c <= maxCallingCount; c = c + 2) { testCheckAndCreateLeasesForNewShardsAtSpecifiedPositionAndClosedShardImpl( ExceptionThrowingLeaseManagerMethods.CREATELEASEIFNOTEXISTS, c, INITIAL_POSITION_TRIM_HORIZON, expectedLeaseKeysToCreate); - // Need to clean up lease manager every time after calling KinesisShardSyncer - leaseManager.deleteAll(); } } @@ -542,6 +541,7 @@ public class ShardSyncerTest { // Only need to try two times. for (int i = 1; i <= 2; i++) { try { + leaseManager.deleteAll(); shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, exceptionThrowingLeaseManager, position, @@ -2412,10 +2412,13 @@ public class ShardSyncerTest { // Make sure ListShardsWithFilter is called in all public shard sync methods shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, false); + + leaseManager.deleteAll(); + shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, false, null); - verify(kinesisProxy, atLeast(2)).getShardListWithFilter(shardFilter); + verify(kinesisProxy, times(2)).getShardListWithFilter(shardFilter); verify(kinesisProxy, never()).getShardList(); } From 49231cc6c1724e74c1ff45085277d816fa7dc98a Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 17 Aug 2020 13:46:41 -0400 Subject: [PATCH 33/43] Adding release notes --- CHANGELOG.md | 48 ++++++++++++++++++++++++++++++++++++++++++- README.md | 58 ++++++++++++++++++++++++++++++++++++---------------- 2 files changed, 87 insertions(+), 19 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 83dcfcce..6f6c8616 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,51 @@ # Changelog -## Latest Release (1.13.3 March 2, 2020) + +## Latest Release (1.14.0 - August 17, 2020) + +* [Milestone#50](https://github.com/awslabs/amazon-kinesis-client/milestone/50) + +* Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. +* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. + * Currently, the following shard filters are supported: + * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. + * `AT_LATEST` - the response includes only the currently open shards of the data stream. + * `AT_TIMESTAMP` - the response includes all shards whose start timestamp is less than or equal to the given timestamp and end timestamp is greater than or equal to the given timestamp or still open. + * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `KinesisClientLibConfiguration#initialPositionInStreamExtended`. + * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). + +* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. + * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. + * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). + +* KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.SHARD_END`, `PeriodicShardSyncManager#leasesRecoveryAuditorInconsistencyConfidenceThreshold` will be used to determine the threshold for number of consecutive scans containing holes in the lease table after which to enforce a shard sync. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.PERIODIC`, `leasesRecoveryAuditorInconsistencyConfidenceThreshold` is ignored. + * New configuration options are available to configure `PeriodicShardSyncManager` in `KinesisClientLibConfiguration` + + | Name | Default | Description | + | ----------------------------------------------------- | ------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | + | leasesRecoveryAuditorInconsistencyConfidenceThreshold | 3 | Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, then it would trigger a shard sync. Only used for `ShardSyncStrategyType.SHARD_END`. | + + * New CloudWatch metrics are also now emitted to monitor the health of `PeriodicShardSyncManager`: + + | Name | Description | + | --------------------------- | ------------------------------------------------------ | + | NumStreamsWithPartialLeases | Number of streams that had holes in their hash ranges. | + | NumStreamsToSync | Number of streams which underwent a full shard sync. | + +* Introducing deferred lease cleanup. Leases will be deleted asynchronously by `LeaseCleanupManager` upon reaching `SHARD_END`, when a shard has either expired past the stream’s retention period or been closed as the result of a resharding operation. + * New configuration options are available to configure `LeaseCleanupManager`. + + | Name | Default | Description | + | ----------------------------------- | ---------- | --------------------------------------------------------------------------------------------------------- | + | leaseCleanupIntervalMillis | 1 minute | Interval at which to run lease cleanup thread. | + | completedLeaseCleanupIntervalMillis | 5 minutes | Interval at which to check if a lease is completed or not. | + | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | + +* Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. +* Upgrading version of AWS SDK to 2.13.X. +* [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. +* [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. + +## Release 1.13.3 (1.13.3 March 2, 2020) [Milestone#49] (https://github.com/awslabs/amazon-kinesis-client/milestone/49) * Refactoring shard closure verification performed by ShutdownTask. * [PR #684] (https://github.com/awslabs/amazon-kinesis-client/pull/684) diff --git a/README.md b/README.md index 8db8b2d7..4279d55e 100644 --- a/README.md +++ b/README.md @@ -31,28 +31,50 @@ To make it easier for developers to write record processors in other languages, ## Release Notes -#### Latest Release (1.13.3 March 2, 2020) -* Refactoring shard closure verification performed by ShutdownTask. - * [PR #684] (https://github.com/awslabs/amazon-kinesis-client/pull/684) -* Fixing the bug in ShardSyncTaskManager to resolve the issue of new shards not being processed after resharding. - * [PR #694] (https://github.com/awslabs/amazon-kinesis-client/pull/694) +### Latest Release (1.14.0 - August 17, 2020) -#### Release (1.13.2 Janurary 13, 2020) -* Adding backward compatible constructors that use the default DDB Billing Mode (#673) - * [PR #673](https://github.com/awslabs/amazon-kinesis-client/pull/673) +* [Milestone#50](https://github.com/awslabs/amazon-kinesis-client/milestone/50) -#### Release (1.13.1 December 30, 2019) -* Adding BillingMode Support to KCL 1.x. This enables the customer to specify if they want provisioned capacity for DDB, or pay per request. - * [PR #656](https://github.com/awslabs/amazon-kinesis-client/pull/656) -* Ensure ShardSyncTask invocation from ShardSyncTaskManager for pending ShardEnd events. - * [PR #659](https://github.com/awslabs/amazon-kinesis-client/pull/659) -* Fix the LeaseManagementIntegrationTest failure. - * [PR #670](https://github.com/awslabs/amazon-kinesis-client/pull/670) +* Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. +* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. + * Currently, the following shard filters are supported: + * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. + * `AT_LATEST` - the response includes only the currently open shards of the data stream. + * `AT_TIMESTAMP` - the response includes all shards whose start timestamp is less than or equal to the given timestamp and end timestamp is greater than or equal to the given timestamp or still open. + * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `KinesisClientLibConfiguration#initialPositionInStreamExtended`. + * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). -#### Release (1.13.0 November 5, 2019) -* Handling completed and blocked tasks better during graceful shutdown - * [PR #640](https://github.com/awslabs/amazon-kinesis-client/pull/640) +* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. + * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. + * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). +* KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.SHARD_END`, `PeriodicShardSyncManager#leasesRecoveryAuditorInconsistencyConfidenceThreshold` will be used to determine the threshold for number of consecutive scans containing holes in the lease table after which to enforce a shard sync. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.PERIODIC`, `leasesRecoveryAuditorInconsistencyConfidenceThreshold` is ignored. + * New configuration options are available to configure `PeriodicShardSyncManager` in `KinesisClientLibConfiguration` + + | Name | Default | Description | + | ----------------------------------------------------- | ------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | + | leasesRecoveryAuditorInconsistencyConfidenceThreshold | 3 | Confidence threshold for the periodic auditor job to determine if leases for a stream in the lease table is inconsistent. If the auditor finds same set of inconsistencies consecutively for a stream for this many times, then it would trigger a shard sync. Only used for `ShardSyncStrategyType.SHARD_END`. | + + * New CloudWatch metrics are also now emitted to monitor the health of `PeriodicShardSyncManager`: + + | Name | Description | + | --------------------------- | ------------------------------------------------------ | + | NumStreamsWithPartialLeases | Number of streams that had holes in their hash ranges. | + | NumStreamsToSync | Number of streams which underwent a full shard sync. | + +* Introducing deferred lease cleanup. Leases will be deleted asynchronously by `LeaseCleanupManager` upon reaching `SHARD_END`, when a shard has either expired past the stream’s retention period or been closed as the result of a resharding operation. + * New configuration options are available to configure `LeaseCleanupManager`. + + | Name | Default | Description | + | ----------------------------------- | ---------- | --------------------------------------------------------------------------------------------------------- | + | leaseCleanupIntervalMillis | 1 minute | Interval at which to run lease cleanup thread. | + | completedLeaseCleanupIntervalMillis | 5 minutes | Interval at which to check if a lease is completed or not. | + | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | + +* Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. +* Upgrading version of AWS SDK to 2.13.X. +* [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. +* [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. ###### For remaining release notes check **[CHANGELOG.md][changelog-md]**. From 7b13527d4111d80971c433b7bd55d053c1cf335b Mon Sep 17 00:00:00 2001 From: Joshua Kim Date: Mon, 17 Aug 2020 15:50:44 -0400 Subject: [PATCH 34/43] More release notes --- CHANGELOG.md | 1 + README.md | 1 + 2 files changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6f6c8616..479d48d9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -41,6 +41,7 @@ | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgr ading. * Upgrading version of AWS SDK to 2.13.X. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. diff --git a/README.md b/README.md index 4279d55e..98b4a31e 100644 --- a/README.md +++ b/README.md @@ -72,6 +72,7 @@ To make it easier for developers to write record processors in other languages, | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgrading. * Upgrading version of AWS SDK to 2.13.X. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. From 566a2446c685ed279d9009c2fdb3a779bc1294b7 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 17 Aug 2020 15:03:40 -0700 Subject: [PATCH 35/43] SDK version upgrade and KCL version update --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 35f8f05c..c256b5be 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.13.4-SNAPSHOT + 1.13.4 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. @@ -25,7 +25,7 @@ - 1.11.728 + 1.11.844 1.0.392 libsqlite4java ${project.build.directory}/test-lib From 2ae76065a8bc71f6ca613d2dfc2e455e6ffc3f1b Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 17 Aug 2020 15:21:48 -0700 Subject: [PATCH 36/43] Correcting the KCL release version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c256b5be..48a802ae 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.13.4 + 1.14.0 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From dd4e88e621773adff40c5b29c5aa3ac4bb83f6dc Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 17 Aug 2020 15:38:55 -0700 Subject: [PATCH 37/43] Updating user agent version --- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 90389a44..643b45d0 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -147,7 +147,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.13.4-SNAPSHOT"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.14.0"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From 9e353db3dbf93abf96907bef7393baaa23e1ff97 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 17 Aug 2020 15:49:42 -0700 Subject: [PATCH 38/43] Updating the SDK version in README and CHANGELOG --- CHANGELOG.md | 2 +- README.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 479d48d9..ec10a8d7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -42,7 +42,7 @@ * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. * Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgr ading. -* Upgrading version of AWS SDK to 2.13.X. +* Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. diff --git a/README.md b/README.md index 98b4a31e..402a1872 100644 --- a/README.md +++ b/README.md @@ -73,7 +73,7 @@ To make it easier for developers to write record processors in other languages, * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. * Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgrading. -* Upgrading version of AWS SDK to 2.13.X. +* Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. From 142415a213960372211cf815890b62979ab63b05 Mon Sep 17 00:00:00 2001 From: Chunxue Yang Date: Mon, 17 Aug 2020 16:44:59 -0700 Subject: [PATCH 39/43] Fixing the readme and changelog docs --- CHANGELOG.md | 10 +++++----- README.md | 8 ++++---- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ec10a8d7..7b1df04a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,7 +5,7 @@ * [Milestone#50](https://github.com/awslabs/amazon-kinesis-client/milestone/50) * Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. -* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. +* When bootstrapping an empty lease table, KCL utilizes the ListShard API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the ShardFilter parameter. The ShardFilter parameter enables you to filter out the response of the ListShards API, using the Type parameter. KCL uses the Type filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. * Currently, the following shard filters are supported: * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. * `AT_LATEST` - the response includes only the currently open shards of the data stream. @@ -13,9 +13,9 @@ * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `KinesisClientLibConfiguration#initialPositionInStreamExtended`. * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). -* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. - * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. - * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). +* Introducing support for the `ChildShards` response of the `GetRecords` API to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. + * For KCL 1.x applications, this uses the `ChildShards` response of the `GetRecords` API. + * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html) and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). * KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.SHARD_END`, `PeriodicShardSyncManager#leasesRecoveryAuditorInconsistencyConfidenceThreshold` will be used to determine the threshold for number of consecutive scans containing holes in the lease table after which to enforce a shard sync. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.PERIODIC`, `leasesRecoveryAuditorInconsistencyConfidenceThreshold` is ignored. * New configuration options are available to configure `PeriodicShardSyncManager` in `KinesisClientLibConfiguration` @@ -41,7 +41,7 @@ | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. -* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgr ading. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgrading. * Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. diff --git a/README.md b/README.md index 402a1872..6fdbff40 100644 --- a/README.md +++ b/README.md @@ -36,7 +36,7 @@ To make it easier for developers to write record processors in other languages, * [Milestone#50](https://github.com/awslabs/amazon-kinesis-client/milestone/50) * Behavior of shard synchronization is moving from each worker independently learning about all existing shards to workers only discovering the children of shards that each worker owns. This optimizes memory usage, lease table IOPS usage, and number of calls made to kinesis for streams with high shard counts and/or frequent resharding. -* When bootstrapping an empty lease table, KCL utilizes the `ListShard` API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the `ShardFilter` parameter. The `ShardFilter` parameter enables you to filter out the response of the `ListShards` API, using the `Type` parameter. KCL uses the `Type` filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. +* When bootstrapping an empty lease table, KCL utilizes the ListShard API's filtering option (the ShardFilter optional request parameter) to retrieve and create leases only for a snapshot of shards open at the time specified by the ShardFilter parameter. The ShardFilter parameter enables you to filter out the response of the ListShards API, using the Type parameter. KCL uses the Type filter parameter and the following of its valid values to identify and return a snapshot of open shards that might require new leases. * Currently, the following shard filters are supported: * `AT_TRIM_HORIZON` - the response includes all the shards that were open at `TRIM_HORIZON`. * `AT_LATEST` - the response includes only the currently open shards of the data stream. @@ -44,9 +44,9 @@ To make it easier for developers to write record processors in other languages, * `ShardFilter` is used when creating leases for an empty lease table to initialize leases for a snapshot of shards specified at `KinesisClientLibConfiguration#initialPositionInStreamExtended`. * For more information about ShardFilter, see the [official AWS documentation on ShardFilter](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ShardFilter.html). -* Introducing support for the `ChildShards` response of the `GetRecords` and the `SubscribeToShard` APIs to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. - * For shared throughout consumer applications, this uses the `ChildShards` response of the `GetRecords` API. For dedicated throughput (enhanced fan-out) consumer applications, this uses the `ChildShards` response of the `SubscribeToShard` API. - * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html), [SubscribeToShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_SubscribeToShard.html), and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). +* Introducing support for the `ChildShards` response of the `GetRecords` API to perform lease/shard synchronization that happens at `SHARD_END` for closed shards, allowing a KCL worker to only create leases for the child shards of the shard it finished processing. + * For KCL 1.x applications, this uses the `ChildShards` response of the `GetRecords` API. + * For more information, see the official AWS Documentation on [GetRecords](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html) and [ChildShard](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ChildShard.html). * KCL now also performs additional periodic shard/lease scans in order to identify any potential holes in the lease table to ensure the complete hash range of the stream is being processed and create leases for them if required. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.SHARD_END`, `PeriodicShardSyncManager#leasesRecoveryAuditorInconsistencyConfidenceThreshold` will be used to determine the threshold for number of consecutive scans containing holes in the lease table after which to enforce a shard sync. When `KinesisClientLibConfiguration#shardSyncStrategyType` is set to `ShardSyncStrategyType.PERIODIC`, `leasesRecoveryAuditorInconsistencyConfidenceThreshold` is ignored. * New configuration options are available to configure `PeriodicShardSyncManager` in `KinesisClientLibConfiguration` From aaf4604357ced1969a30daf6bf1d8728c5ec4eff Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 17 Aug 2020 19:56:52 -0400 Subject: [PATCH 40/43] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7b1df04a..05ea98ed 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -41,7 +41,7 @@ | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. -* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgrading. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstPrioritization` while upgrading. * Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. From 1440eeabbc34d55d5ee321623063828ba98e3343 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 17 Aug 2020 19:57:40 -0400 Subject: [PATCH 41/43] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 6fdbff40..cb71cbe7 100644 --- a/README.md +++ b/README.md @@ -72,7 +72,7 @@ To make it easier for developers to write record processors in other languages, | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. -* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentFirstPrioritization` while upgrading. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstPrioritization` while upgrading. * Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. From 53174940e86d5e3df81b7ce82653e150bfba4bee Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 17 Aug 2020 19:58:13 -0400 Subject: [PATCH 42/43] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index cb71cbe7..f936d602 100644 --- a/README.md +++ b/README.md @@ -72,7 +72,7 @@ To make it easier for developers to write record processors in other languages, | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. -* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstPrioritization` while upgrading. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstShardPrioritization` while upgrading. * Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy. From ff004d75be17831f80dc6e58296a183823ed9904 Mon Sep 17 00:00:00 2001 From: Joshua Kim <20001595+joshua-kim@users.noreply.github.com> Date: Mon, 17 Aug 2020 19:58:35 -0400 Subject: [PATCH 43/43] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 05ea98ed..3188791a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -41,7 +41,7 @@ | garbageLeaseCleanupIntervalMillis | 30 minutes | Interval at which to check if a lease is garbage (i.e trimmed past the stream's retention period) or not. | * Including an optimization to `KinesisShardSyncer` to only create leases for one layer of shards. -* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstPrioritization` while upgrading. +* Changing default shard prioritization strategy to be `NoOpShardPrioritization` to allow prioritization of completed shards. Customers who are upgrading to this version and are reading from `TRIM_HORIZON` should continue using `ParentsFirstShardPrioritization` while upgrading. * Upgrading version of AWS SDK to 1.11.844. * [#719](https://github.com/awslabs/amazon-kinesis-client/pull/719) Upgrading version of Google Protobuf to 3.11.4. * [#712](https://github.com/awslabs/amazon-kinesis-client/pull/712) Allowing KCL to consider lease tables in `UPDATING` healthy.