allow unexpected child shards to be ignored
now instead of always throwing an assertion if a child shard has an open parent, consider worker configuration before doing so. if configured to ignore such shards, do not create leases for them during shard sync. this is intended to mitigate failing worker init when processing dynamodb streams with many thousands of shards (which can happen for tables with thousands of partitions). this new behavior can be enabled by adding the following to a configuration/properties file: ``` ignoreUnexpectedChildShards = true ```
This commit is contained in:
parent
9720b1b249
commit
2afb0bd00b
13 changed files with 299 additions and 158 deletions
|
|
@ -515,7 +515,9 @@ class ConsumerStates {
|
|||
consumer.getRecordProcessorCheckpointer(), consumer.getShutdownReason(),
|
||||
consumer.getStreamConfig().getStreamProxy(),
|
||||
consumer.getStreamConfig().getInitialPositionInStream(),
|
||||
consumer.isCleanupLeasesOfCompletedShards(), consumer.getLeaseManager(),
|
||||
consumer.isCleanupLeasesOfCompletedShards(),
|
||||
consumer.isIgnoreUnexpectedChildShards(),
|
||||
consumer.getLeaseManager(),
|
||||
consumer.getTaskBackoffTimeMillis(),
|
||||
consumer.getGetRecordsRetrievalStrategy());
|
||||
}
|
||||
|
|
|
|||
|
|
@ -86,6 +86,11 @@ public class KinesisClientLibConfiguration {
|
|||
*/
|
||||
public static final boolean DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION = true;
|
||||
|
||||
/**
|
||||
* Ignore child shards with open parents.
|
||||
*/
|
||||
public static final boolean DEFAULT_IGNORE_UNEXPECTED_CHILD_SHARDS = false;
|
||||
|
||||
/**
|
||||
* Backoff time in milliseconds for Amazon Kinesis Client Library tasks (in the event of failures).
|
||||
*/
|
||||
|
|
@ -200,6 +205,7 @@ public class KinesisClientLibConfiguration {
|
|||
private boolean callProcessRecordsEvenForEmptyRecordList;
|
||||
private long parentShardPollIntervalMillis;
|
||||
private boolean cleanupLeasesUponShardCompletion;
|
||||
private boolean ignoreUnexpectedChildShards;
|
||||
private ClientConfiguration kinesisClientConfig;
|
||||
private ClientConfiguration dynamoDBClientConfig;
|
||||
private ClientConfiguration cloudWatchClientConfig;
|
||||
|
|
@ -272,6 +278,7 @@ public class KinesisClientLibConfiguration {
|
|||
DEFAULT_MAX_RECORDS, DEFAULT_IDLETIME_BETWEEN_READS_MILLIS,
|
||||
DEFAULT_DONT_CALL_PROCESS_RECORDS_FOR_EMPTY_RECORD_LIST, DEFAULT_PARENT_SHARD_POLL_INTERVAL_MILLIS,
|
||||
DEFAULT_SHARD_SYNC_INTERVAL_MILLIS, DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION,
|
||||
DEFAULT_IGNORE_UNEXPECTED_CHILD_SHARDS,
|
||||
new ClientConfiguration(), new ClientConfiguration(), new ClientConfiguration(),
|
||||
DEFAULT_TASK_BACKOFF_TIME_MILLIS, DEFAULT_METRICS_BUFFER_TIME_MILLIS, DEFAULT_METRICS_MAX_QUEUE_SIZE,
|
||||
DEFAULT_VALIDATE_SEQUENCE_NUMBER_BEFORE_CHECKPOINTING, null,
|
||||
|
|
@ -300,6 +307,7 @@ public class KinesisClientLibConfiguration {
|
|||
* @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 ignoreUnexpectedChildShards Ignore child shards with open parents
|
||||
* @param kinesisClientConfig Client Configuration used by Kinesis client
|
||||
* @param dynamoDBClientConfig Client Configuration used by DynamoDB client
|
||||
* @param cloudWatchClientConfig Client Configuration used by CloudWatch client
|
||||
|
|
@ -329,6 +337,7 @@ public class KinesisClientLibConfiguration {
|
|||
long parentShardPollIntervalMillis,
|
||||
long shardSyncIntervalMillis,
|
||||
boolean cleanupTerminatedShardsBeforeExpiry,
|
||||
boolean ignoreUnexpectedChildShards,
|
||||
ClientConfiguration kinesisClientConfig,
|
||||
ClientConfiguration dynamoDBClientConfig,
|
||||
ClientConfiguration cloudWatchClientConfig,
|
||||
|
|
@ -342,7 +351,7 @@ public class KinesisClientLibConfiguration {
|
|||
dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId,
|
||||
maxRecords, idleTimeBetweenReadsInMillis,
|
||||
callProcessRecordsEvenForEmptyRecordList, parentShardPollIntervalMillis,
|
||||
shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry,
|
||||
shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry, ignoreUnexpectedChildShards,
|
||||
kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig,
|
||||
taskBackoffTimeMillis, metricsBufferTimeMillis, metricsMaxQueueSize,
|
||||
validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis);
|
||||
|
|
@ -371,6 +380,7 @@ public class KinesisClientLibConfiguration {
|
|||
* @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 ignoreUnexpectedChildShards Ignore child shards with open parents
|
||||
* @param kinesisClientConfig Client Configuration used by Kinesis client
|
||||
* @param dynamoDBClientConfig Client Configuration used by DynamoDB client
|
||||
* @param cloudWatchClientConfig Client Configuration used by CloudWatch client
|
||||
|
|
@ -400,6 +410,7 @@ public class KinesisClientLibConfiguration {
|
|||
long parentShardPollIntervalMillis,
|
||||
long shardSyncIntervalMillis,
|
||||
boolean cleanupTerminatedShardsBeforeExpiry,
|
||||
boolean ignoreUnexpectedChildShards,
|
||||
ClientConfiguration kinesisClientConfig,
|
||||
ClientConfiguration dynamoDBClientConfig,
|
||||
ClientConfiguration cloudWatchClientConfig,
|
||||
|
|
@ -436,6 +447,7 @@ public class KinesisClientLibConfiguration {
|
|||
this.parentShardPollIntervalMillis = parentShardPollIntervalMillis;
|
||||
this.shardSyncIntervalMillis = shardSyncIntervalMillis;
|
||||
this.cleanupLeasesUponShardCompletion = cleanupTerminatedShardsBeforeExpiry;
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
this.workerIdentifier = workerId;
|
||||
this.kinesisClientConfig = checkAndAppendKinesisClientLibUserAgent(kinesisClientConfig);
|
||||
this.dynamoDBClientConfig = checkAndAppendKinesisClientLibUserAgent(dynamoDBClientConfig);
|
||||
|
|
@ -670,6 +682,13 @@ public class KinesisClientLibConfiguration {
|
|||
return cleanupLeasesUponShardCompletion;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if we should ignore child shards which have open parents
|
||||
*/
|
||||
public boolean shouldIgnoreUnexpectedChildShards() {
|
||||
return ignoreUnexpectedChildShards;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if KCL should validate client provided sequence numbers with a call to Amazon Kinesis before
|
||||
* checkpointing for calls to {@link RecordProcessorCheckpointer#checkpoint(String)}
|
||||
|
|
@ -890,6 +909,16 @@ public class KinesisClientLibConfiguration {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param ignoreUnexpectedChildShards Ignore child shards with open parents.
|
||||
* @return KinesisClientLibConfiguration
|
||||
*/
|
||||
public KinesisClientLibConfiguration withIgnoreUnexpectedChildShards(
|
||||
boolean ignoreUnexpectedChildShards) {
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientConfig Common client configuration used by Kinesis/DynamoDB/CloudWatch client
|
||||
* @return KinesisClientLibConfiguration
|
||||
|
|
|
|||
|
|
@ -53,6 +53,7 @@ class ShardConsumer {
|
|||
// Backoff time when polling to check if application has finished processing parent shards
|
||||
private final long parentShardPollIntervalMillis;
|
||||
private final boolean cleanupLeasesOfCompletedShards;
|
||||
private final boolean ignoreUnexpectedChildShards;
|
||||
private final long taskBackoffTimeMillis;
|
||||
private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
|
||||
|
|
@ -97,7 +98,7 @@ class ShardConsumer {
|
|||
* @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard
|
||||
* @param backoffTimeMillis backoff interval when we encounter exceptions
|
||||
*/
|
||||
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
|
||||
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 11 LINES
|
||||
ShardConsumer(ShardInfo shardInfo,
|
||||
StreamConfig streamConfig,
|
||||
ICheckpoint checkpoint,
|
||||
|
|
@ -105,12 +106,13 @@ class ShardConsumer {
|
|||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
long parentShardPollIntervalMillis,
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
boolean ignoreUnexpectedChildShards,
|
||||
ExecutorService executorService,
|
||||
IMetricsFactory metricsFactory,
|
||||
long backoffTimeMillis,
|
||||
boolean skipShardSyncAtWorkerInitializationIfLeasesExist) {
|
||||
this(shardInfo, streamConfig, checkpoint,recordProcessor, leaseManager, parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards, executorService, metricsFactory, backoffTimeMillis,
|
||||
cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards, executorService, metricsFactory, backoffTimeMillis,
|
||||
skipShardSyncAtWorkerInitializationIfLeasesExist, Optional.empty(), Optional.empty());
|
||||
}
|
||||
|
||||
|
|
@ -135,6 +137,7 @@ class ShardConsumer {
|
|||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
long parentShardPollIntervalMillis,
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
boolean ignoreUnexpectedChildShards,
|
||||
ExecutorService executorService,
|
||||
IMetricsFactory metricsFactory,
|
||||
long backoffTimeMillis,
|
||||
|
|
@ -157,6 +160,7 @@ class ShardConsumer {
|
|||
this.metricsFactory = metricsFactory;
|
||||
this.parentShardPollIntervalMillis = parentShardPollIntervalMillis;
|
||||
this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards;
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
this.taskBackoffTimeMillis = backoffTimeMillis;
|
||||
this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
this.getRecordsRetrievalStrategy = makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo);
|
||||
|
|
@ -402,6 +406,10 @@ class ShardConsumer {
|
|||
return cleanupLeasesOfCompletedShards;
|
||||
}
|
||||
|
||||
boolean isIgnoreUnexpectedChildShards() {
|
||||
return ignoreUnexpectedChildShards;
|
||||
}
|
||||
|
||||
long getTaskBackoffTimeMillis() {
|
||||
return taskBackoffTimeMillis;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -35,6 +35,7 @@ class ShardSyncTask implements ITask {
|
|||
private final ILeaseManager<KinesisClientLease> leaseManager;
|
||||
private InitialPositionInStreamExtended initialPosition;
|
||||
private final boolean cleanupLeasesUponShardCompletion;
|
||||
private final boolean ignoreUnexpectedChildShards;
|
||||
private final long shardSyncTaskIdleTimeMillis;
|
||||
private final TaskType taskType = TaskType.SHARDSYNC;
|
||||
|
||||
|
|
@ -49,11 +50,13 @@ class ShardSyncTask implements ITask {
|
|||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
InitialPositionInStreamExtended initialPositionInStream,
|
||||
boolean cleanupLeasesUponShardCompletion,
|
||||
boolean ignoreUnexpectedChildShards,
|
||||
long shardSyncTaskIdleTimeMillis) {
|
||||
this.kinesisProxy = kinesisProxy;
|
||||
this.leaseManager = leaseManager;
|
||||
this.initialPosition = initialPositionInStream;
|
||||
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
this.shardSyncTaskIdleTimeMillis = shardSyncTaskIdleTimeMillis;
|
||||
}
|
||||
|
||||
|
|
@ -68,7 +71,8 @@ class ShardSyncTask implements ITask {
|
|||
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy,
|
||||
leaseManager,
|
||||
initialPosition,
|
||||
cleanupLeasesUponShardCompletion);
|
||||
cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards);
|
||||
if (shardSyncTaskIdleTimeMillis > 0) {
|
||||
Thread.sleep(shardSyncTaskIdleTimeMillis);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -44,6 +44,7 @@ class ShardSyncTaskManager {
|
|||
private final ExecutorService executorService;
|
||||
private final InitialPositionInStreamExtended initialPositionInStream;
|
||||
private boolean cleanupLeasesUponShardCompletion;
|
||||
private boolean ignoreUnexpectedChildShards;
|
||||
private final long shardSyncIdleTimeMillis;
|
||||
|
||||
|
||||
|
|
@ -55,6 +56,7 @@ class ShardSyncTaskManager {
|
|||
* @param initialPositionInStream Initial position in stream
|
||||
* @param cleanupLeasesUponShardCompletion Clean up leases for shards that we've finished processing (don't wait
|
||||
* until they expire)
|
||||
* @param ignoreUnexpectedChildShards Ignore child shards with open parents
|
||||
* @param shardSyncIdleTimeMillis Time between tasks to sync leases and Kinesis shards
|
||||
* @param metricsFactory Metrics factory
|
||||
* @param executorService ExecutorService to execute the shard sync tasks
|
||||
|
|
@ -63,6 +65,7 @@ class ShardSyncTaskManager {
|
|||
final ILeaseManager<KinesisClientLease> leaseManager,
|
||||
final InitialPositionInStreamExtended initialPositionInStream,
|
||||
final boolean cleanupLeasesUponShardCompletion,
|
||||
final boolean ignoreUnexpectedChildShards,
|
||||
final long shardSyncIdleTimeMillis,
|
||||
final IMetricsFactory metricsFactory,
|
||||
ExecutorService executorService) {
|
||||
|
|
@ -70,6 +73,7 @@ class ShardSyncTaskManager {
|
|||
this.leaseManager = leaseManager;
|
||||
this.metricsFactory = metricsFactory;
|
||||
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis;
|
||||
this.executorService = executorService;
|
||||
this.initialPositionInStream = initialPositionInStream;
|
||||
|
|
@ -99,6 +103,7 @@ class ShardSyncTaskManager {
|
|||
leaseManager,
|
||||
initialPositionInStream,
|
||||
cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards,
|
||||
shardSyncIdleTimeMillis), metricsFactory);
|
||||
future = executorService.submit(currentTask);
|
||||
submittedNewTask = true;
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ class ShardSyncer {
|
|||
InitialPositionInStreamExtended initialPositionInStream,
|
||||
boolean cleanupLeasesOfCompletedShards)
|
||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
|
||||
syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards);
|
||||
syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -71,21 +71,28 @@ class ShardSyncer {
|
|||
* @param kinesisProxy
|
||||
* @param leaseManager
|
||||
* @param initialPositionInStream
|
||||
* @param expectedClosedShardId If this is not null, we will assert that the shard list we get from Kinesis
|
||||
* shows this shard to be closed (e.g. parent shard must be closed after a reshard operation).
|
||||
* If it is open, we assume this is an race condition around a reshard event and throw
|
||||
* a KinesisClientLibIOException so client can backoff and retry later.
|
||||
* @param cleanupLeasesOfCompletedShards
|
||||
* @param ignoreUnexpectedChildShards
|
||||
* @throws DependencyException
|
||||
* @throws InvalidStateException
|
||||
* @throws ProvisionedThroughputException
|
||||
* @throws KinesisClientLibIOException
|
||||
*/
|
||||
static synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy,
|
||||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
InitialPositionInStreamExtended initialPositionInStream,
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
boolean ignoreUnexpectedChildShards)
|
||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
|
||||
syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards);
|
||||
}
|
||||
|
||||
static synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy,
|
||||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
InitialPositionInStreamExtended initialPositionInStream,
|
||||
boolean cleanupLeasesOfCompletedShards)
|
||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
|
||||
syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards);
|
||||
checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -93,11 +100,9 @@ class ShardSyncer {
|
|||
*
|
||||
* @param kinesisProxy
|
||||
* @param leaseManager
|
||||
* @param expectedClosedShardId If this is not null, we will assert that the shard list we get from Kinesis
|
||||
* does not show this shard to be open (e.g. parent shard must be closed after a reshard operation).
|
||||
* If it is still open, we assume this is a race condition around a reshard event and
|
||||
* throw a KinesisClientLibIOException so client can backoff and retry later. If the shard doesn't exist in
|
||||
* Kinesis at all, we assume this is an old/expired shard and continue with the sync operation.
|
||||
* @param initialPosition
|
||||
* @param cleanupLeasesOfCompletedShards
|
||||
* @param ignoreUnexpectedChildShards
|
||||
* @throws DependencyException
|
||||
* @throws InvalidStateException
|
||||
* @throws ProvisionedThroughputException
|
||||
|
|
@ -107,18 +112,23 @@ class ShardSyncer {
|
|||
private static synchronized void syncShardLeases(IKinesisProxy kinesisProxy,
|
||||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
InitialPositionInStreamExtended initialPosition,
|
||||
boolean cleanupLeasesOfCompletedShards)
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
boolean ignoreUnexpectedChildShards)
|
||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
|
||||
List<Shard> shards = getShardList(kinesisProxy);
|
||||
LOG.debug("Num shards: " + shards.size());
|
||||
|
||||
Map<String, Shard> shardIdToShardMap = constructShardIdToShardMap(shards);
|
||||
Map<String, Set<String>> shardIdToChildShardIdsMap = constructShardIdToChildShardIdsMap(shardIdToShardMap);
|
||||
assertAllParentShardsAreClosed(shardIdToChildShardIdsMap, shardIdToShardMap);
|
||||
Set<String> inconsistentShardIds = findInconsistentShardIds(shardIdToChildShardIdsMap, shardIdToShardMap);
|
||||
if (!ignoreUnexpectedChildShards) {
|
||||
assertAllParentShardsAreClosed(inconsistentShardIds);
|
||||
}
|
||||
|
||||
List<KinesisClientLease> currentLeases = leaseManager.listLeases();
|
||||
|
||||
List<KinesisClientLease> newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition);
|
||||
List<KinesisClientLease> newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition,
|
||||
inconsistentShardIds);
|
||||
LOG.debug("Num new leases to create: " + newLeasesToCreate.size());
|
||||
for (KinesisClientLease lease : newLeasesToCreate) {
|
||||
long startTimeMillis = System.currentTimeMillis();
|
||||
|
|
@ -149,19 +159,39 @@ class ShardSyncer {
|
|||
|
||||
/** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls
|
||||
* and a reshard operation.
|
||||
* @param shardIdToChildShardIdsMap
|
||||
* @param shardIdToShardMap
|
||||
* @param inconsistentShardIds
|
||||
* @throws KinesisClientLibIOException
|
||||
*/
|
||||
private static void assertAllParentShardsAreClosed(Map<String, Set<String>> shardIdToChildShardIdsMap,
|
||||
Map<String, Shard> shardIdToShardMap) throws KinesisClientLibIOException {
|
||||
for (String parentShardId : shardIdToChildShardIdsMap.keySet()) {
|
||||
Shard parentShard = shardIdToShardMap.get(parentShardId);
|
||||
if ((parentShardId == null) || (parentShard.getSequenceNumberRange().getEndingSequenceNumber() == null)) {
|
||||
throw new KinesisClientLibIOException("Parent shardId " + parentShardId + " is not closed. "
|
||||
private static void assertAllParentShardsAreClosed(Set<String> inconsistentShardIds)
|
||||
throws KinesisClientLibIOException {
|
||||
if (!inconsistentShardIds.isEmpty()) {
|
||||
String ids = "";
|
||||
for (String id : inconsistentShardIds) {
|
||||
ids += " " + id;
|
||||
}
|
||||
throw new KinesisClientLibIOException(String.valueOf(inconsistentShardIds.size()) + " open child shards (" + ids + ") are inconsistent."
|
||||
+ "This can happen due to a race condition between describeStream and a reshard operation.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to construct the list of inconsistent shards, which are open shards with non-closed ancestor
|
||||
* parent(s).
|
||||
* @param shardIdToChildShardIdsMap
|
||||
* @param shardIdToShardMap
|
||||
* @return Set of inconsistent open shard ids for shards having open parents.
|
||||
*/
|
||||
private static Set<String> findInconsistentShardIds(Map<String, Set<String>> shardIdToChildShardIdsMap,
|
||||
Map<String, Shard> shardIdToShardMap) {
|
||||
Set<String> result = new HashSet<String>();
|
||||
for (String parentShardId : shardIdToChildShardIdsMap.keySet()) {
|
||||
Shard parentShard = shardIdToShardMap.get(parentShardId);
|
||||
if ((parentShardId == null) || (parentShard.getSequenceNumberRange().getEndingSequenceNumber() == null)) {
|
||||
Set<String> childShardIdsMap = shardIdToChildShardIdsMap.get(parentShardId);
|
||||
result.addAll(childShardIdsMap);
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -297,7 +327,7 @@ class 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 that doesn't already have a lease,
|
||||
* 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.
|
||||
|
|
@ -327,15 +357,22 @@ class ShardSyncer {
|
|||
* 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
|
||||
*/
|
||||
static List<KinesisClientLease> determineNewLeasesToCreate(List<Shard> shards,
|
||||
List<KinesisClientLease> currentLeases,
|
||||
InitialPositionInStreamExtended initialPosition) {
|
||||
InitialPositionInStreamExtended initialPosition,
|
||||
Set<String> inconsistentShardIds) {
|
||||
Map<String, KinesisClientLease> shardIdToNewLeaseMap = new HashMap<String, KinesisClientLease>();
|
||||
Map<String, Shard> shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards);
|
||||
|
||||
|
|
@ -354,6 +391,8 @@ class ShardSyncer {
|
|||
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);
|
||||
|
|
@ -407,6 +446,17 @@ class ShardSyncer {
|
|||
return newLeasesToCreate;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine new leases to create and their initial checkpoint.
|
||||
* Note: Package level access only for testing purposes.
|
||||
*/
|
||||
static List<KinesisClientLease> determineNewLeasesToCreate(List<Shard> shards,
|
||||
List<KinesisClientLease> currentLeases,
|
||||
InitialPositionInStreamExtended initialPosition) {
|
||||
Set<String> inconsistentShardIds = new HashSet<String>();
|
||||
return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds);
|
||||
}
|
||||
|
||||
/**
|
||||
* Note: Package level access for testing purposes only.
|
||||
* Check if this shard is a descendant of a shard that is (or will be) processed.
|
||||
|
|
|
|||
|
|
@ -44,6 +44,7 @@ class ShutdownTask implements ITask {
|
|||
private final ILeaseManager<KinesisClientLease> leaseManager;
|
||||
private final InitialPositionInStreamExtended initialPositionInStream;
|
||||
private final boolean cleanupLeasesOfCompletedShards;
|
||||
private final boolean ignoreUnexpectedChildShards;
|
||||
private final TaskType taskType = TaskType.SHUTDOWN;
|
||||
private final long backoffTimeMillis;
|
||||
private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
|
@ -59,6 +60,7 @@ class ShutdownTask implements ITask {
|
|||
IKinesisProxy kinesisProxy,
|
||||
InitialPositionInStreamExtended initialPositionInStream,
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
boolean ignoreUnexpectedChildShards,
|
||||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
long backoffTimeMillis,
|
||||
GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) {
|
||||
|
|
@ -69,6 +71,7 @@ class ShutdownTask implements ITask {
|
|||
this.kinesisProxy = kinesisProxy;
|
||||
this.initialPositionInStream = initialPositionInStream;
|
||||
this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards;
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
this.leaseManager = leaseManager;
|
||||
this.backoffTimeMillis = backoffTimeMillis;
|
||||
this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy;
|
||||
|
|
@ -127,7 +130,8 @@ class ShutdownTask implements ITask {
|
|||
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy,
|
||||
leaseManager,
|
||||
initialPositionInStream,
|
||||
cleanupLeasesOfCompletedShards);
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards);
|
||||
LOG.debug("Finished checking for child shards of shard " + shardInfo.getShardId());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -103,6 +103,7 @@ public class Worker implements Runnable {
|
|||
// info, value is ShardConsumer.
|
||||
private ConcurrentMap<ShardInfo, ShardConsumer> shardInfoShardConsumerMap = new ConcurrentHashMap<ShardInfo, ShardConsumer>();
|
||||
private final boolean cleanupLeasesUponShardCompletion;
|
||||
private final boolean ignoreUnexpectedChildShards;
|
||||
|
||||
private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
|
||||
|
|
@ -253,7 +254,8 @@ public class Worker implements Runnable {
|
|||
config.shouldValidateSequenceNumberBeforeCheckpointing(),
|
||||
config.getInitialPositionInStreamExtended()),
|
||||
config.getInitialPositionInStreamExtended(), config.getParentShardPollIntervalMillis(),
|
||||
config.getShardSyncIntervalMillis(), config.shouldCleanupLeasesUponShardCompletion(), null,
|
||||
config.getShardSyncIntervalMillis(), config.shouldCleanupLeasesUponShardCompletion(),
|
||||
config.shouldIgnoreUnexpectedChildShards(), null,
|
||||
new KinesisClientLibLeaseCoordinator(
|
||||
new KinesisClientLeaseManager(config.getTableName(), dynamoDBClient),
|
||||
config.getWorkerIdentifier(),
|
||||
|
|
@ -318,6 +320,8 @@ public class Worker implements Runnable {
|
|||
* Time between tasks to sync leases and Kinesis shards
|
||||
* @param cleanupLeasesUponShardCompletion
|
||||
* Clean up shards we've finished processing (don't wait till they expire in Kinesis)
|
||||
* @param ignoreUnexpectedChildShards
|
||||
* Ignore child shards with open parents
|
||||
* @param checkpoint
|
||||
* Used to get/set checkpoints
|
||||
* @param leaseCoordinator
|
||||
|
|
@ -335,14 +339,14 @@ public class Worker implements Runnable {
|
|||
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
|
||||
Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, StreamConfig streamConfig,
|
||||
InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis,
|
||||
long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint,
|
||||
KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
|
||||
long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, boolean ignoreUnexpectedChildShards,
|
||||
ICheckpoint checkpoint, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
|
||||
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
|
||||
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization) {
|
||||
this(applicationName, recordProcessorFactory, streamConfig, initialPositionInStream, parentShardPollIntervalMillis,
|
||||
shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService,
|
||||
metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
shardPrioritization, Optional.empty(), Optional.empty());
|
||||
shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, checkpoint,
|
||||
leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis,
|
||||
skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization, Optional.empty(), Optional.empty());
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -363,6 +367,8 @@ public class Worker implements Runnable {
|
|||
* Time between tasks to sync leases and Kinesis shards
|
||||
* @param cleanupLeasesUponShardCompletion
|
||||
* Clean up shards we've finished processing (don't wait till they expire in Kinesis)
|
||||
* @param ignoreUnexpectedChildShards
|
||||
* Ignore child shards with open parents
|
||||
* @param checkpoint
|
||||
* Used to get/set checkpoints
|
||||
* @param leaseCoordinator
|
||||
|
|
@ -384,8 +390,8 @@ public class Worker implements Runnable {
|
|||
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
|
||||
Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, StreamConfig streamConfig,
|
||||
InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis,
|
||||
long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint,
|
||||
KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
|
||||
long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, boolean ignoreUnexpectedChildShards,
|
||||
ICheckpoint checkpoint, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
|
||||
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
|
||||
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization,
|
||||
Optional<Integer> retryGetRecordsInSeconds, Optional<Integer> maxGetRecordsThreadPool) {
|
||||
|
|
@ -395,14 +401,15 @@ public class Worker implements Runnable {
|
|||
this.initialPosition = initialPositionInStream;
|
||||
this.parentShardPollIntervalMillis = parentShardPollIntervalMillis;
|
||||
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
|
||||
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
|
||||
this.checkpointTracker = checkpoint != null ? checkpoint : leaseCoordinator;
|
||||
this.idleTimeInMilliseconds = streamConfig.getIdleTimeInMilliseconds();
|
||||
this.executorService = execService;
|
||||
this.leaseCoordinator = leaseCoordinator;
|
||||
this.metricsFactory = metricsFactory;
|
||||
this.controlServer = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(),
|
||||
initialPositionInStream, cleanupLeasesUponShardCompletion, shardSyncIdleTimeMillis, metricsFactory,
|
||||
executorService);
|
||||
initialPositionInStream, cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, shardSyncIdleTimeMillis,
|
||||
metricsFactory, executorService);
|
||||
this.taskBackoffTimeMillis = taskBackoffTimeMillis;
|
||||
this.failoverTimeMillis = failoverTimeMillis;
|
||||
this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
|
|
@ -494,7 +501,8 @@ public class Worker implements Runnable {
|
|||
|| leaseCoordinator.getLeaseManager().isLeaseTableEmpty()) {
|
||||
LOG.info("Syncing Kinesis shard info");
|
||||
ShardSyncTask shardSyncTask = new ShardSyncTask(streamConfig.getStreamProxy(),
|
||||
leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion, 0L);
|
||||
leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards, 0L);
|
||||
result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call();
|
||||
} else {
|
||||
LOG.info("Skipping shard sync per config setting (and lease table is not empty)");
|
||||
|
|
@ -844,7 +852,7 @@ public class Worker implements Runnable {
|
|||
|
||||
return new ShardConsumer(shardInfo, streamConfig, checkpointTracker, recordProcessor,
|
||||
leaseCoordinator.getLeaseManager(), parentShardPollIntervalMillis, cleanupLeasesUponShardCompletion,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis,
|
||||
ignoreUnexpectedChildShards, executorService, metricsFactory, taskBackoffTimeMillis,
|
||||
skipShardSyncAtWorkerInitializationIfLeasesExist, retryGetRecordsInSeconds, maxGetRecordsThreadPool);
|
||||
|
||||
}
|
||||
|
|
@ -1255,6 +1263,7 @@ public class Worker implements Runnable {
|
|||
config.getParentShardPollIntervalMillis(),
|
||||
config.getShardSyncIntervalMillis(),
|
||||
config.shouldCleanupLeasesUponShardCompletion(),
|
||||
config.shouldIgnoreUnexpectedChildShards(),
|
||||
null,
|
||||
new KinesisClientLibLeaseCoordinator(new KinesisClientLeaseManager(config.getTableName(),
|
||||
dynamoDBClient),
|
||||
|
|
|
|||
|
|
@ -77,6 +77,7 @@ public class KinesisClientLibConfigurationTest {
|
|||
TEST_VALUE_LONG,
|
||||
TEST_VALUE_LONG,
|
||||
true,
|
||||
false,
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
|
|
@ -116,6 +117,7 @@ public class KinesisClientLibConfigurationTest {
|
|||
longValues[2],
|
||||
longValues[3],
|
||||
true,
|
||||
false,
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
|
|
@ -151,6 +153,7 @@ public class KinesisClientLibConfigurationTest {
|
|||
TEST_VALUE_LONG,
|
||||
TEST_VALUE_LONG,
|
||||
true,
|
||||
false,
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
|
|
@ -315,6 +318,7 @@ public class KinesisClientLibConfigurationTest {
|
|||
TEST_VALUE_LONG,
|
||||
TEST_VALUE_LONG,
|
||||
true,
|
||||
false,
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
new ClientConfiguration(),
|
||||
|
|
|
|||
|
|
@ -87,6 +87,7 @@ public class ShardConsumerTest {
|
|||
private final long taskBackoffTimeMillis = 500L;
|
||||
private final long parentShardPollIntervalMillis = 50L;
|
||||
private final boolean cleanupLeasesOfCompletedShards = true;
|
||||
private final boolean ignoreUnexpectedChildShards = false;
|
||||
// We don't want any of these tests to run checkpoint validation
|
||||
private final boolean skipCheckpointValidationValue = false;
|
||||
private static final InitialPositionInStreamExtended INITIAL_POSITION_LATEST =
|
||||
|
|
@ -134,6 +135,7 @@ public class ShardConsumerTest {
|
|||
null,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -182,6 +184,7 @@ public class ShardConsumerTest {
|
|||
null,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
spyExecutorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -223,6 +226,7 @@ public class ShardConsumerTest {
|
|||
null,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -318,6 +322,7 @@ public class ShardConsumerTest {
|
|||
leaseManager,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -421,6 +426,7 @@ public class ShardConsumerTest {
|
|||
leaseManager,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -483,6 +489,7 @@ public class ShardConsumerTest {
|
|||
null,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -530,6 +537,7 @@ public class ShardConsumerTest {
|
|||
null,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
@ -558,6 +566,7 @@ public class ShardConsumerTest {
|
|||
null,
|
||||
parentShardPollIntervalMillis,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
executorService,
|
||||
metricsFactory,
|
||||
taskBackoffTimeMillis,
|
||||
|
|
|
|||
|
|
@ -123,7 +123,7 @@ public class ShardSyncTaskIntegrationTest {
|
|||
ShardSyncTask syncTask = new ShardSyncTask(kinesisProxy,
|
||||
leaseManager,
|
||||
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST),
|
||||
false,
|
||||
false, false,
|
||||
0L);
|
||||
syncTask.call();
|
||||
List<KinesisClientLease> leases = leaseManager.listLeases();
|
||||
|
|
|
|||
|
|
@ -100,6 +100,7 @@ public class ShutdownTaskTest {
|
|||
IKinesisProxy kinesisProxy = mock(IKinesisProxy.class);
|
||||
ILeaseManager<KinesisClientLease> leaseManager = mock(KinesisClientLeaseManager.class);
|
||||
boolean cleanupLeasesOfCompletedShards = false;
|
||||
boolean ignoreUnexpectedChildShards = false;
|
||||
ShutdownTask task = new ShutdownTask(defaultShardInfo,
|
||||
defaultRecordProcessor,
|
||||
checkpointer,
|
||||
|
|
@ -107,6 +108,7 @@ public class ShutdownTaskTest {
|
|||
kinesisProxy,
|
||||
INITIAL_POSITION_TRIM_HORIZON,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
leaseManager,
|
||||
TASK_BACKOFF_TIME_MILLIS,
|
||||
getRecordsRetrievalStrategy);
|
||||
|
|
@ -126,6 +128,7 @@ public class ShutdownTaskTest {
|
|||
when(kinesisProxy.getShardList()).thenReturn(null);
|
||||
ILeaseManager<KinesisClientLease> leaseManager = mock(KinesisClientLeaseManager.class);
|
||||
boolean cleanupLeasesOfCompletedShards = false;
|
||||
boolean ignoreUnexpectedChildShards = false;
|
||||
ShutdownTask task = new ShutdownTask(defaultShardInfo,
|
||||
defaultRecordProcessor,
|
||||
checkpointer,
|
||||
|
|
@ -133,6 +136,7 @@ public class ShutdownTaskTest {
|
|||
kinesisProxy,
|
||||
INITIAL_POSITION_TRIM_HORIZON,
|
||||
cleanupLeasesOfCompletedShards,
|
||||
ignoreUnexpectedChildShards,
|
||||
leaseManager,
|
||||
TASK_BACKOFF_TIME_MILLIS,
|
||||
getRecordsRetrievalStrategy);
|
||||
|
|
@ -147,7 +151,7 @@ public class ShutdownTaskTest {
|
|||
*/
|
||||
@Test
|
||||
public final void testGetTaskType() {
|
||||
ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, null, 0, getRecordsRetrievalStrategy);
|
||||
ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, null, 0, getRecordsRetrievalStrategy);
|
||||
Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -119,6 +119,7 @@ public class WorkerTest {
|
|||
private final long parentShardPollIntervalMillis = 5L;
|
||||
private final long shardSyncIntervalMillis = 5L;
|
||||
private final boolean cleanupLeasesUponShardCompletion = true;
|
||||
private final boolean ignoreUnexpectedChildShards = false;
|
||||
// We don't want any of these tests to run checkpoint validation
|
||||
private final boolean skipCheckpointValidationValue = false;
|
||||
private static final InitialPositionInStreamExtended INITIAL_POSITION_LATEST =
|
||||
|
|
@ -232,6 +233,7 @@ public class WorkerTest {
|
|||
parentShardPollIntervalMillis,
|
||||
shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards,
|
||||
checkpoint,
|
||||
leaseCoordinator,
|
||||
execService,
|
||||
|
|
@ -276,8 +278,9 @@ public class WorkerTest {
|
|||
.thenReturn(secondCheckpoint);
|
||||
|
||||
Worker worker = new Worker(stageName, streamletFactory, streamConfig, INITIAL_POSITION_LATEST,
|
||||
parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupLeasesUponShardCompletion, checkpoint,
|
||||
leaseCoordinator, execService, nullMetricsFactory, taskBackoffTimeMillis, failoverTimeMillis,
|
||||
parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards, checkpoint, leaseCoordinator, execService, nullMetricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis,
|
||||
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, shardPrioritization);
|
||||
|
||||
Worker workerSpy = spy(worker);
|
||||
|
|
@ -336,6 +339,7 @@ public class WorkerTest {
|
|||
parentShardPollIntervalMillis,
|
||||
shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards,
|
||||
checkpoint,
|
||||
leaseCoordinator,
|
||||
execService,
|
||||
|
|
@ -390,6 +394,7 @@ public class WorkerTest {
|
|||
shardPollInterval,
|
||||
shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards,
|
||||
leaseCoordinator,
|
||||
leaseCoordinator,
|
||||
execService,
|
||||
|
|
@ -744,8 +749,9 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization);
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false,
|
||||
shardPrioritization);
|
||||
|
||||
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
|
||||
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
|
||||
|
|
@ -818,8 +824,8 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) {
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) {
|
||||
@Override
|
||||
void postConstruct() {
|
||||
this.gracefuleShutdownStarted = true;
|
||||
|
|
@ -890,8 +896,8 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) {
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) {
|
||||
@Override
|
||||
void postConstruct() {
|
||||
this.gracefulShutdownCoordinator = coordinator;
|
||||
|
|
@ -952,8 +958,9 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization);
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false,
|
||||
shardPrioritization);
|
||||
|
||||
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
|
||||
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
|
||||
|
|
@ -1022,8 +1029,9 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization);
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false,
|
||||
shardPrioritization);
|
||||
|
||||
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
|
||||
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
|
||||
|
|
@ -1123,8 +1131,9 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization);
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false,
|
||||
shardPrioritization);
|
||||
|
||||
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
|
||||
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
|
||||
|
|
@ -1228,8 +1237,9 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization);
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false,
|
||||
shardPrioritization);
|
||||
|
||||
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
|
||||
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
|
||||
|
|
@ -1300,8 +1310,9 @@ public class WorkerTest {
|
|||
|
||||
Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig,
|
||||
INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization);
|
||||
cleanupLeasesUponShardCompletion, ignoreUnexpectedChildShards, leaseCoordinator, leaseCoordinator,
|
||||
executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false,
|
||||
shardPrioritization);
|
||||
|
||||
when(executorService.submit(Matchers.<Callable<TaskResult>> any()))
|
||||
.thenAnswer(new ShutdownHandlingAnswer(taskFuture));
|
||||
|
|
@ -1338,14 +1349,15 @@ public class WorkerTest {
|
|||
InjectableWorker(String applicationName, IRecordProcessorFactory recordProcessorFactory,
|
||||
StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream,
|
||||
long parentShardPollIntervalMillis, long shardSyncIdleTimeMillis,
|
||||
boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint,
|
||||
boolean cleanupLeasesUponShardCompletion, boolean ignoreUnexpectedChildShards, ICheckpoint checkpoint,
|
||||
KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
|
||||
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
|
||||
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization) {
|
||||
super(applicationName, recordProcessorFactory, streamConfig, initialPositionInStream,
|
||||
parentShardPollIntervalMillis, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion,
|
||||
checkpoint, leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis,
|
||||
failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization);
|
||||
ignoreUnexpectedChildShards, checkpoint, leaseCoordinator, execService, metricsFactory,
|
||||
taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
shardPrioritization);
|
||||
postConstruct();
|
||||
}
|
||||
|
||||
|
|
@ -1657,6 +1669,7 @@ public class WorkerTest {
|
|||
parentShardPollIntervalMillis,
|
||||
shardSyncIntervalMillis,
|
||||
cleanupLeasesUponShardCompletion,
|
||||
ignoreUnexpectedChildShards,
|
||||
leaseCoordinator,
|
||||
leaseCoordinator,
|
||||
executorService,
|
||||
|
|
|
|||
Loading…
Reference in a new issue