Separated out some lease taker logic as interface implementations to … (#490)

Added interfaces to allow external users to control the lease selection, and cleanup.
This commit is contained in:
achitojha 2019-02-20 11:48:40 -08:00 committed by Justin Pfifer
parent 54e6a48a48
commit fbdd449759
23 changed files with 811 additions and 528 deletions

View file

@ -530,7 +530,8 @@ class ConsumerStates {
consumer.isIgnoreUnexpectedChildShards(), consumer.isIgnoreUnexpectedChildShards(),
consumer.getLeaseManager(), consumer.getLeaseManager(),
consumer.getTaskBackoffTimeMillis(), consumer.getTaskBackoffTimeMillis(),
consumer.getGetRecordsCache()); consumer.getGetRecordsCache(),
consumer.getShardSyncer());
} }
@Override @Override

View file

@ -22,6 +22,8 @@ import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.UUID; import java.util.UUID;
import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -50,6 +52,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
private static final long DEFAULT_INITIAL_LEASE_TABLE_READ_CAPACITY = 10L; private static final long DEFAULT_INITIAL_LEASE_TABLE_READ_CAPACITY = 10L;
private static final long DEFAULT_INITIAL_LEASE_TABLE_WRITE_CAPACITY = 10L; private static final long DEFAULT_INITIAL_LEASE_TABLE_WRITE_CAPACITY = 10L;
private static final LeaseSelector<KinesisClientLease> DEFAULT_LEASE_SELECTOR = new GenericLeaseSelector<KinesisClientLease>();
private final ILeaseManager<KinesisClientLease> leaseManager; private final ILeaseManager<KinesisClientLease> leaseManager;
@ -61,12 +64,14 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
* @param workerIdentifier Used to identify this worker process * @param workerIdentifier Used to identify this worker process
* @param leaseDurationMillis Duration of a lease in milliseconds * @param leaseDurationMillis Duration of a lease in milliseconds
* @param epsilonMillis Delta for timing operations (e.g. checking lease expiry) * @param epsilonMillis Delta for timing operations (e.g. checking lease expiry)
* @param leaseSelector Lease selector which decides which leases to take
*/ */
public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager, public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager,
String workerIdentifier, String workerIdentifier,
long leaseDurationMillis, long leaseDurationMillis,
long epsilonMillis) { long epsilonMillis,
super(leaseManager, workerIdentifier, leaseDurationMillis, epsilonMillis); LeaseSelector<KinesisClientLease> leaseSelector) {
super(leaseManager, leaseSelector, workerIdentifier, leaseDurationMillis, epsilonMillis);
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
} }
@ -75,19 +80,35 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
* @param workerIdentifier Used to identify this worker process * @param workerIdentifier Used to identify this worker process
* @param leaseDurationMillis Duration of a lease in milliseconds * @param leaseDurationMillis Duration of a lease in milliseconds
* @param epsilonMillis Delta for timing operations (e.g. checking lease expiry) * @param epsilonMillis Delta for timing operations (e.g. checking lease expiry)
* @param metricsFactory Metrics factory used to emit metrics
*/ */
public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager, public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager,
String workerIdentifier,
long leaseDurationMillis,
long epsilonMillis) {
this(leaseManager, workerIdentifier, leaseDurationMillis, epsilonMillis, DEFAULT_LEASE_SELECTOR);
}
/**
* @param leaseManager Lease manager which provides CRUD lease operations.
* @param leaseSelector Lease selector which decides which leases to take
* @param workerIdentifier Used to identify this worker process
* @param leaseDurationMillis Duration of a lease in milliseconds
* @param epsilonMillis Delta for timing operations (e.g. checking lease expiry)
* @param metricsFactory Metrics factory used to emit metrics
*/
public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager,
LeaseSelector<KinesisClientLease> leaseSelector,
String workerIdentifier, String workerIdentifier,
long leaseDurationMillis, long leaseDurationMillis,
long epsilonMillis, long epsilonMillis,
IMetricsFactory metricsFactory) { IMetricsFactory metricsFactory) {
super(leaseManager, workerIdentifier, leaseDurationMillis, epsilonMillis, metricsFactory); super(leaseManager, leaseSelector, workerIdentifier, leaseDurationMillis, epsilonMillis, metricsFactory);
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
} }
/** /**
* @param leaseManager Lease manager which provides CRUD lease operations. * @param leaseManager Lease manager which provides CRUD lease operations.
* @param leaseSelector Lease selector which decides which leases to take
* @param workerIdentifier Used to identify this worker process * @param workerIdentifier Used to identify this worker process
* @param leaseDurationMillis Duration of a lease in milliseconds * @param leaseDurationMillis Duration of a lease in milliseconds
* @param epsilonMillis Delta for timing operations (e.g. checking lease expiry) * @param epsilonMillis Delta for timing operations (e.g. checking lease expiry)
@ -96,6 +117,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
* @param metricsFactory Metrics factory used to emit metrics * @param metricsFactory Metrics factory used to emit metrics
*/ */
public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager, public KinesisClientLibLeaseCoordinator(ILeaseManager<KinesisClientLease> leaseManager,
LeaseSelector<KinesisClientLease> leaseSelector,
String workerIdentifier, String workerIdentifier,
long leaseDurationMillis, long leaseDurationMillis,
long epsilonMillis, long epsilonMillis,
@ -103,7 +125,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
int maxLeasesToStealAtOneTime, int maxLeasesToStealAtOneTime,
int maxLeaseRenewerThreadCount, int maxLeaseRenewerThreadCount,
IMetricsFactory metricsFactory) { IMetricsFactory metricsFactory) {
super(leaseManager, workerIdentifier, leaseDurationMillis, epsilonMillis, maxLeasesForWorker, super(leaseManager, leaseSelector, workerIdentifier, leaseDurationMillis, epsilonMillis, maxLeasesForWorker,
maxLeasesToStealAtOneTime, maxLeaseRenewerThreadCount, metricsFactory); maxLeasesToStealAtOneTime, maxLeaseRenewerThreadCount, metricsFactory);
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
} }

View file

@ -0,0 +1,50 @@
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.KinesisClientLibIOException;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.Set;
/**
* Represents the class that decides if a lease is eligible for cleanup.
*/
class KinesisLeaseCleanupValidator implements LeaseCleanupValidator {
private static final Log LOG = LogFactory.getLog(KinesisLeaseCleanupValidator.class);
/**
* @param lease Candidate shard we are considering for deletion.
* @param currentKinesisShardIds
* @return true if neither the shard (corresponding to the lease), nor its parents are present in
* currentKinesisShardIds
* @throws KinesisClientLibIOException Thrown if currentKinesisShardIds contains a parent shard but not the child
* shard (we are evaluating for deletion).
*/
@Override
public boolean isCandidateForCleanup(KinesisClientLease lease, Set<String> currentKinesisShardIds) throws KinesisClientLibIOException {
boolean isCandidateForCleanup = true;
if (currentKinesisShardIds.contains(lease.getLeaseKey())) {
isCandidateForCleanup = false;
} else {
LOG.info("Found lease for non-existent shard: " + lease.getLeaseKey() + ". Checking its parent shards");
Set<String> parentShardIds = lease.getParentShardIds();
for (String parentShardId : parentShardIds) {
// Throw an exception if the parent shard exists (but the child does not).
// This may be a (rare) race condition between fetching the shard list and Kinesis expiring shards.
if (currentKinesisShardIds.contains(parentShardId)) {
String message =
"Parent shard " + parentShardId + " exists but not the child shard "
+ lease.getLeaseKey();
LOG.info(message);
throw new KinesisClientLibIOException(message);
}
}
}
return isCandidateForCleanup;
}
}

View file

@ -0,0 +1,21 @@
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.KinesisClientLibIOException;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
import java.util.Set;
/**
* Represents the class that decides if a lease is eligible for cleanup.
*/
public interface LeaseCleanupValidator {
/**
* @param lease Candidate shard we are considering for deletion.
* @param currentKinesisShardIds
* @return boolean representing if the lease is eligible for cleanup.
* @throws KinesisClientLibIOException
*/
boolean isCandidateForCleanup(KinesisClientLease lease, Set<String> currentKinesisShardIds)
throws KinesisClientLibIOException;
}

View file

@ -58,6 +58,9 @@ class ShardConsumer {
private final long taskBackoffTimeMillis; private final long taskBackoffTimeMillis;
private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist;
@Getter
private final ShardSyncer shardSyncer;
private ITask currentTask; private ITask currentTask;
private long currentTaskSubmitTime; private long currentTaskSubmitTime;
private Future<TaskResult> future; private Future<TaskResult> future;
@ -99,6 +102,7 @@ class ShardConsumer {
* @param executorService ExecutorService used to execute process tasks for this shard * @param executorService ExecutorService used to execute process tasks for this shard
* @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard * @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard
* @param backoffTimeMillis backoff interval when we encounter exceptions * @param backoffTimeMillis backoff interval when we encounter exceptions
* @param shardSyncer shardSyncer instance used to check and create new leases
*/ */
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
ShardConsumer(ShardInfo shardInfo, ShardConsumer(ShardInfo shardInfo,
@ -112,7 +116,8 @@ class ShardConsumer {
IMetricsFactory metricsFactory, IMetricsFactory metricsFactory,
long backoffTimeMillis, long backoffTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
KinesisClientLibConfiguration config) { KinesisClientLibConfiguration config,
ShardSyncer shardSyncer) {
this(shardInfo, this(shardInfo,
streamConfig, streamConfig,
checkpoint, checkpoint,
@ -126,7 +131,8 @@ class ShardConsumer {
skipShardSyncAtWorkerInitializationIfLeasesExist, skipShardSyncAtWorkerInitializationIfLeasesExist,
Optional.empty(), Optional.empty(),
Optional.empty(), Optional.empty(),
config); config,
shardSyncer);
} }
/** /**
@ -142,6 +148,7 @@ class ShardConsumer {
* @param retryGetRecordsInSeconds time in seconds to wait before the worker retries to get a record. * @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 maxGetRecordsThreadPool max number of threads in the getRecords thread pool.
* @param config Kinesis library configuration * @param config Kinesis library configuration
* @param shardSyncer shardSyncer instance used to check and create new leases
*/ */
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
ShardConsumer(ShardInfo shardInfo, ShardConsumer(ShardInfo shardInfo,
@ -157,7 +164,8 @@ class ShardConsumer {
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
Optional<Integer> retryGetRecordsInSeconds, Optional<Integer> retryGetRecordsInSeconds,
Optional<Integer> maxGetRecordsThreadPool, Optional<Integer> maxGetRecordsThreadPool,
KinesisClientLibConfiguration config) { KinesisClientLibConfiguration config,
ShardSyncer shardSyncer) {
this( this(
shardInfo, shardInfo,
@ -182,7 +190,8 @@ class ShardConsumer {
new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo), new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo),
retryGetRecordsInSeconds, retryGetRecordsInSeconds,
maxGetRecordsThreadPool, maxGetRecordsThreadPool,
config config,
shardSyncer
); );
} }
@ -203,6 +212,7 @@ class ShardConsumer {
* @param retryGetRecordsInSeconds time in seconds to wait before the worker retries to get a record * @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 maxGetRecordsThreadPool max number of threads in the getRecords thread pool
* @param config Kinesis library configuration * @param config Kinesis library configuration
* @param shardSyncer shardSyncer instance used to check and create new leases
*/ */
ShardConsumer(ShardInfo shardInfo, ShardConsumer(ShardInfo shardInfo,
StreamConfig streamConfig, StreamConfig streamConfig,
@ -219,7 +229,8 @@ class ShardConsumer {
KinesisDataFetcher kinesisDataFetcher, KinesisDataFetcher kinesisDataFetcher,
Optional<Integer> retryGetRecordsInSeconds, Optional<Integer> retryGetRecordsInSeconds,
Optional<Integer> maxGetRecordsThreadPool, Optional<Integer> maxGetRecordsThreadPool,
KinesisClientLibConfiguration config) { KinesisClientLibConfiguration config,
ShardSyncer shardSyncer) {
this.shardInfo = shardInfo; this.shardInfo = shardInfo;
this.streamConfig = streamConfig; this.streamConfig = streamConfig;
this.checkpoint = checkpoint; this.checkpoint = checkpoint;
@ -237,6 +248,7 @@ class ShardConsumer {
this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher( this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher(
makeStrategy(this.dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, this.shardInfo), makeStrategy(this.dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, this.shardInfo),
this.getShardInfo().getShardId(), this.metricsFactory, this.config.getMaxRecords()); this.getShardInfo().getShardId(), this.metricsFactory, this.config.getMaxRecords());
this.shardSyncer = shardSyncer;
} }
/** /**

View file

@ -38,6 +38,7 @@ class ShardSyncTask implements ITask {
private final boolean ignoreUnexpectedChildShards; private final boolean ignoreUnexpectedChildShards;
private final long shardSyncTaskIdleTimeMillis; private final long shardSyncTaskIdleTimeMillis;
private final TaskType taskType = TaskType.SHARDSYNC; private final TaskType taskType = TaskType.SHARDSYNC;
private final ShardSyncer shardSyncer;
/** /**
* @param kinesisProxy Used to fetch information about the stream (e.g. shard list) * @param kinesisProxy Used to fetch information about the stream (e.g. shard list)
@ -45,19 +46,25 @@ class ShardSyncTask implements ITask {
* @param initialPositionInStream One of LATEST, TRIM_HORIZON or AT_TIMESTAMP. Amazon Kinesis Client Library will * @param initialPositionInStream One of LATEST, TRIM_HORIZON or AT_TIMESTAMP. Amazon Kinesis Client Library will
* start processing records from this point in the stream (when an application starts up for the first time) * start processing records from this point in the stream (when an application starts up for the first time)
* except for shards that already have a checkpoint (and their descendant shards). * except for shards that already have a checkpoint (and their descendant shards).
* @param cleanupLeasesUponShardCompletion Clean up shards we've finished processing (don't wait for expiration
* in Kinesis)
* @param shardSyncTaskIdleTimeMillis shardSync task idle time in millis
* @param shardSyncer shardSyncer instance used to check and create new leases
*/ */
ShardSyncTask(IKinesisProxy kinesisProxy, ShardSyncTask(IKinesisProxy kinesisProxy,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
InitialPositionInStreamExtended initialPositionInStream, InitialPositionInStreamExtended initialPositionInStream,
boolean cleanupLeasesUponShardCompletion, boolean cleanupLeasesUponShardCompletion,
boolean ignoreUnexpectedChildShards, boolean ignoreUnexpectedChildShards,
long shardSyncTaskIdleTimeMillis) { long shardSyncTaskIdleTimeMillis,
ShardSyncer shardSyncer) {
this.kinesisProxy = kinesisProxy; this.kinesisProxy = kinesisProxy;
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
this.initialPosition = initialPositionInStream; this.initialPosition = initialPositionInStream;
this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion;
this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards;
this.shardSyncTaskIdleTimeMillis = shardSyncTaskIdleTimeMillis; this.shardSyncTaskIdleTimeMillis = shardSyncTaskIdleTimeMillis;
this.shardSyncer = shardSyncer;
} }
/* (non-Javadoc) /* (non-Javadoc)
@ -68,7 +75,7 @@ class ShardSyncTask implements ITask {
Exception exception = null; Exception exception = null;
try { try {
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy,
leaseManager, leaseManager,
initialPosition, initialPosition,
cleanupLeasesUponShardCompletion, cleanupLeasesUponShardCompletion,

View file

@ -46,6 +46,7 @@ class ShardSyncTaskManager {
private boolean cleanupLeasesUponShardCompletion; private boolean cleanupLeasesUponShardCompletion;
private boolean ignoreUnexpectedChildShards; private boolean ignoreUnexpectedChildShards;
private final long shardSyncIdleTimeMillis; private final long shardSyncIdleTimeMillis;
private final ShardSyncer shardSyncer;
/** /**
@ -60,6 +61,7 @@ class ShardSyncTaskManager {
* @param shardSyncIdleTimeMillis Time between tasks to sync leases and Kinesis shards * @param shardSyncIdleTimeMillis Time between tasks to sync leases and Kinesis shards
* @param metricsFactory Metrics factory * @param metricsFactory Metrics factory
* @param executorService ExecutorService to execute the shard sync tasks * @param executorService ExecutorService to execute the shard sync tasks
* @param shardSyncer shardSyncer instance used to check and create new leases
*/ */
ShardSyncTaskManager(final IKinesisProxy kinesisProxy, ShardSyncTaskManager(final IKinesisProxy kinesisProxy,
final ILeaseManager<KinesisClientLease> leaseManager, final ILeaseManager<KinesisClientLease> leaseManager,
@ -68,7 +70,8 @@ class ShardSyncTaskManager {
final boolean ignoreUnexpectedChildShards, final boolean ignoreUnexpectedChildShards,
final long shardSyncIdleTimeMillis, final long shardSyncIdleTimeMillis,
final IMetricsFactory metricsFactory, final IMetricsFactory metricsFactory,
ExecutorService executorService) { ExecutorService executorService,
ShardSyncer shardSyncer) {
this.kinesisProxy = kinesisProxy; this.kinesisProxy = kinesisProxy;
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
this.metricsFactory = metricsFactory; this.metricsFactory = metricsFactory;
@ -77,6 +80,7 @@ class ShardSyncTaskManager {
this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis; this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis;
this.executorService = executorService; this.executorService = executorService;
this.initialPositionInStream = initialPositionInStream; this.initialPositionInStream = initialPositionInStream;
this.shardSyncer = shardSyncer;
} }
synchronized boolean syncShardAndLeaseInfo(Set<String> closedShardIds) { synchronized boolean syncShardAndLeaseInfo(Set<String> closedShardIds) {
@ -104,7 +108,8 @@ class ShardSyncTaskManager {
initialPositionInStream, initialPositionInStream,
cleanupLeasesUponShardCompletion, cleanupLeasesUponShardCompletion,
ignoreUnexpectedChildShards, ignoreUnexpectedChildShards,
shardSyncIdleTimeMillis), metricsFactory); shardSyncIdleTimeMillis,
shardSyncer), metricsFactory);
future = executorService.submit(currentTask); future = executorService.submit(currentTask);
submittedNewTask = true; submittedNewTask = true;
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {

View file

@ -51,14 +51,13 @@ import com.amazonaws.services.kinesis.model.Shard;
class ShardSyncer { class ShardSyncer {
private static final Log LOG = LogFactory.getLog(ShardSyncer.class); private static final Log LOG = LogFactory.getLog(ShardSyncer.class);
private final LeaseCleanupValidator leaseCleanupValidator;
/** public ShardSyncer(final LeaseCleanupValidator leaseCleanupValidator) {
* Note constructor is private: We use static synchronized methods - this is a utility class. this.leaseCleanupValidator = leaseCleanupValidator;
*/
private ShardSyncer() {
} }
static synchronized void bootstrapShardLeases(IKinesisProxy kinesisProxy, synchronized void bootstrapShardLeases(IKinesisProxy kinesisProxy,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
InitialPositionInStreamExtended initialPositionInStream, InitialPositionInStreamExtended initialPositionInStream,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
@ -81,7 +80,7 @@ class ShardSyncer {
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
static synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy, synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
InitialPositionInStreamExtended initialPositionInStream, InitialPositionInStreamExtended initialPositionInStream,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
@ -90,14 +89,6 @@ class ShardSyncer {
syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards); syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards);
} }
static synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy,
ILeaseManager<KinesisClientLease> leaseManager,
InitialPositionInStreamExtended initialPositionInStream,
boolean cleanupLeasesOfCompletedShards)
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, false);
}
/** /**
* Sync leases with Kinesis shards (e.g. at startup, or when we reach end of a shard). * Sync leases with Kinesis shards (e.g. at startup, or when we reach end of a shard).
* *
@ -112,7 +103,7 @@ class ShardSyncer {
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
// CHECKSTYLE:OFF CyclomaticComplexity // CHECKSTYLE:OFF CyclomaticComplexity
private static synchronized void syncShardLeases(IKinesisProxy kinesisProxy, private synchronized void syncShardLeases(IKinesisProxy kinesisProxy,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
InitialPositionInStreamExtended initialPosition, InitialPositionInStreamExtended initialPosition,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
@ -165,7 +156,7 @@ class ShardSyncer {
* @param inconsistentShardIds * @param inconsistentShardIds
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
private static void assertAllParentShardsAreClosed(Set<String> inconsistentShardIds) private void assertAllParentShardsAreClosed(Set<String> inconsistentShardIds)
throws KinesisClientLibIOException { throws KinesisClientLibIOException {
if (!inconsistentShardIds.isEmpty()) { if (!inconsistentShardIds.isEmpty()) {
String ids = StringUtils.join(inconsistentShardIds, ' '); String ids = StringUtils.join(inconsistentShardIds, ' ');
@ -182,7 +173,7 @@ class ShardSyncer {
* @param shardIdToShardMap * @param shardIdToShardMap
* @return Set of inconsistent open shard ids for shards having open parents. * @return Set of inconsistent open shard ids for shards having open parents.
*/ */
private static Set<String> findInconsistentShardIds(Map<String, Set<String>> shardIdToChildShardIdsMap, private Set<String> findInconsistentShardIds(Map<String, Set<String>> shardIdToChildShardIdsMap,
Map<String, Shard> shardIdToShardMap) { Map<String, Shard> shardIdToShardMap) {
Set<String> result = new HashSet<String>(); Set<String> result = new HashSet<String>();
for (String parentShardId : shardIdToChildShardIdsMap.keySet()) { for (String parentShardId : shardIdToChildShardIdsMap.keySet()) {
@ -201,7 +192,7 @@ class ShardSyncer {
* @param trackedLeaseList * @param trackedLeaseList
* @return * @return
*/ */
static Map<String, KinesisClientLease> constructShardIdToKCLLeaseMap(List<KinesisClientLease> trackedLeaseList) { Map<String, KinesisClientLease> constructShardIdToKCLLeaseMap(List<KinesisClientLease> trackedLeaseList) {
Map<String, KinesisClientLease> trackedLeasesMap = new HashMap<>(); Map<String, KinesisClientLease> trackedLeasesMap = new HashMap<>();
for (KinesisClientLease lease : trackedLeaseList) { for (KinesisClientLease lease : trackedLeaseList) {
trackedLeasesMap.put(lease.getLeaseKey(), lease); trackedLeasesMap.put(lease.getLeaseKey(), lease);
@ -214,12 +205,8 @@ class ShardSyncer {
* Useful for asserting that we don't have an incomplete shard list following a reshard operation. * Useful for asserting that we don't have an incomplete shard list following a reshard operation.
* We verify that if the shard is present in the shard list, it is closed and its hash key range * We verify that if the shard is present in the shard list, it is closed and its hash key range
* is covered by its child shards. * is covered by its child shards.
* @param shards List of all Kinesis shards
* @param shardIdsOfClosedShards Id of the shard which is expected to be closed
* @return ShardIds of child shards (children of the expectedClosedShard)
* @throws KinesisClientLibIOException
*/ */
static synchronized void assertClosedShardsAreCoveredOrAbsent(Map<String, Shard> shardIdToShardMap, synchronized void assertClosedShardsAreCoveredOrAbsent(Map<String, Shard> shardIdToShardMap,
Map<String, Set<String>> shardIdToChildShardIdsMap, Map<String, Set<String>> shardIdToChildShardIdsMap,
Set<String> shardIdsOfClosedShards) throws KinesisClientLibIOException { Set<String> shardIdsOfClosedShards) throws KinesisClientLibIOException {
String exceptionMessageSuffix = "This can happen if we constructed the list of shards " String exceptionMessageSuffix = "This can happen if we constructed the list of shards "
@ -248,7 +235,7 @@ class ShardSyncer {
} }
} }
private static synchronized void assertHashRangeOfClosedShardIsCovered(Shard closedShard, private synchronized void assertHashRangeOfClosedShardIsCovered(Shard closedShard,
Map<String, Shard> shardIdToShardMap, Map<String, Shard> shardIdToShardMap,
Set<String> childShardIds) throws KinesisClientLibIOException { Set<String> childShardIds) throws KinesisClientLibIOException {
@ -286,7 +273,7 @@ class ShardSyncer {
* @param shardIdToShardMap * @param shardIdToShardMap
* @return * @return
*/ */
static Map<String, Set<String>> constructShardIdToChildShardIdsMap( Map<String, Set<String>> constructShardIdToChildShardIdsMap(
Map<String, Shard> shardIdToShardMap) { Map<String, Shard> shardIdToShardMap) {
Map<String, Set<String>> shardIdToChildShardIdsMap = new HashMap<>(); Map<String, Set<String>> shardIdToChildShardIdsMap = new HashMap<>();
for (Map.Entry<String, Shard> entry : shardIdToShardMap.entrySet()) { for (Map.Entry<String, Shard> entry : shardIdToShardMap.entrySet()) {
@ -315,7 +302,7 @@ class ShardSyncer {
return shardIdToChildShardIdsMap; return shardIdToChildShardIdsMap;
} }
private static List<Shard> getShardList(IKinesisProxy kinesisProxy) throws KinesisClientLibIOException { private List<Shard> getShardList(IKinesisProxy kinesisProxy) throws KinesisClientLibIOException {
List<Shard> shards = kinesisProxy.getShardList(); List<Shard> shards = kinesisProxy.getShardList();
if (shards == null) { if (shards == null) {
throw new KinesisClientLibIOException( throw new KinesisClientLibIOException(
@ -371,7 +358,7 @@ class ShardSyncer {
* @param inconsistentShardIds Set of child shard ids having open parents. * @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 * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard
*/ */
static List<KinesisClientLease> determineNewLeasesToCreate(List<Shard> shards, List<KinesisClientLease> determineNewLeasesToCreate(List<Shard> shards,
List<KinesisClientLease> currentLeases, List<KinesisClientLease> currentLeases,
InitialPositionInStreamExtended initialPosition, InitialPositionInStreamExtended initialPosition,
Set<String> inconsistentShardIds) { Set<String> inconsistentShardIds) {
@ -452,7 +439,7 @@ class ShardSyncer {
* Determine new leases to create and their initial checkpoint. * Determine new leases to create and their initial checkpoint.
* Note: Package level access only for testing purposes. * Note: Package level access only for testing purposes.
*/ */
static List<KinesisClientLease> determineNewLeasesToCreate(List<Shard> shards, List<KinesisClientLease> determineNewLeasesToCreate(List<Shard> shards,
List<KinesisClientLease> currentLeases, List<KinesisClientLease> currentLeases,
InitialPositionInStreamExtended initialPosition) { InitialPositionInStreamExtended initialPosition) {
Set<String> inconsistentShardIds = new HashSet<String>(); Set<String> inconsistentShardIds = new HashSet<String>();
@ -475,7 +462,7 @@ class ShardSyncer {
* @return true if the shard is a descendant of any current shard (lease already exists) * @return true if the shard is a descendant of any current shard (lease already exists)
*/ */
// CHECKSTYLE:OFF CyclomaticComplexity // CHECKSTYLE:OFF CyclomaticComplexity
static boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId, boolean checkIfDescendantAndAddNewLeasesForAncestors(String shardId,
InitialPositionInStreamExtended initialPosition, InitialPositionInStreamExtended initialPosition,
Set<String> shardIdsOfCurrentLeases, Set<String> shardIdsOfCurrentLeases,
Map<String, Shard> shardIdToShardMapOfAllKinesisShards, Map<String, Shard> shardIdToShardMapOfAllKinesisShards,
@ -566,7 +553,7 @@ class ShardSyncer {
* @param shardIdToShardMapOfAllKinesisShards ShardId->Shard map containing all shards obtained via DescribeStream. * @param shardIdToShardMapOfAllKinesisShards ShardId->Shard map containing all shards obtained via DescribeStream.
* @return Set of parentShardIds * @return Set of parentShardIds
*/ */
static Set<String> getParentShardIds(Shard shard, Map<String, Shard> shardIdToShardMapOfAllKinesisShards) { Set<String> getParentShardIds(Shard shard, Map<String, Shard> shardIdToShardMapOfAllKinesisShards) {
Set<String> parentShardIds = new HashSet<String>(2); Set<String> parentShardIds = new HashSet<String>(2);
String parentShardId = shard.getParentShardId(); String parentShardId = shard.getParentShardId();
if ((parentShardId != null) && shardIdToShardMapOfAllKinesisShards.containsKey(parentShardId)) { if ((parentShardId != null) && shardIdToShardMapOfAllKinesisShards.containsKey(parentShardId)) {
@ -593,7 +580,7 @@ class ShardSyncer {
* @throws InvalidStateException * @throws InvalidStateException
* @throws DependencyException * @throws DependencyException
*/ */
private static void cleanupGarbageLeases(List<Shard> shards, private void cleanupGarbageLeases(List<Shard> shards,
List<KinesisClientLease> trackedLeases, List<KinesisClientLease> trackedLeases,
IKinesisProxy kinesisProxy, IKinesisProxy kinesisProxy,
ILeaseManager<KinesisClientLease> leaseManager) ILeaseManager<KinesisClientLease> leaseManager)
@ -606,7 +593,7 @@ class ShardSyncer {
// Check if there are leases for non-existent shards // Check if there are leases for non-existent shards
List<KinesisClientLease> garbageLeases = new ArrayList<>(); List<KinesisClientLease> garbageLeases = new ArrayList<>();
for (KinesisClientLease lease : trackedLeases) { for (KinesisClientLease lease : trackedLeases) {
if (isCandidateForCleanup(lease, kinesisShards)) { if (leaseCleanupValidator.isCandidateForCleanup(lease, kinesisShards)) {
garbageLeases.add(lease); garbageLeases.add(lease);
} }
} }
@ -622,7 +609,7 @@ class ShardSyncer {
} }
for (KinesisClientLease lease : garbageLeases) { for (KinesisClientLease lease : garbageLeases) {
if (isCandidateForCleanup(lease, currentKinesisShardIds)) { if (leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds)) {
LOG.info("Deleting lease for shard " + lease.getLeaseKey() LOG.info("Deleting lease for shard " + lease.getLeaseKey()
+ " as it is not present in Kinesis stream."); + " as it is not present in Kinesis stream.");
leaseManager.deleteLease(lease); leaseManager.deleteLease(lease);
@ -632,42 +619,6 @@ class ShardSyncer {
} }
/**
* Note: This method has package level access, solely for testing purposes.
*
* @param lease Candidate shard we are considering for deletion.
* @param currentKinesisShardIds
* @return true if neither the shard (corresponding to the lease), nor its parents are present in
* currentKinesisShardIds
* @throws KinesisClientLibIOException Thrown if currentKinesisShardIds contains a parent shard but not the child
* shard (we are evaluating for deletion).
*/
static boolean isCandidateForCleanup(KinesisClientLease lease, Set<String> currentKinesisShardIds)
throws KinesisClientLibIOException {
boolean isCandidateForCleanup = true;
if (currentKinesisShardIds.contains(lease.getLeaseKey())) {
isCandidateForCleanup = false;
} else {
LOG.info("Found lease for non-existent shard: " + lease.getLeaseKey() + ". Checking its parent shards");
Set<String> parentShardIds = lease.getParentShardIds();
for (String parentShardId : parentShardIds) {
// Throw an exception if the parent shard exists (but the child does not).
// This may be a (rare) race condition between fetching the shard list and Kinesis expiring shards.
if (currentKinesisShardIds.contains(parentShardId)) {
String message =
"Parent shard " + parentShardId + " exists but not the child shard "
+ lease.getLeaseKey();
LOG.info(message);
throw new KinesisClientLibIOException(message);
}
}
}
return isCandidateForCleanup;
}
/** /**
* Private helper method. * Private helper method.
* Clean up leases for shards that meet the following criteria: * Clean up leases for shards that meet the following criteria:
@ -685,7 +636,7 @@ class ShardSyncer {
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
private static synchronized void cleanupLeasesOfFinishedShards(Collection<KinesisClientLease> currentLeases, private synchronized void cleanupLeasesOfFinishedShards(Collection<KinesisClientLease> currentLeases,
Map<String, Shard> shardIdToShardMap, Map<String, Shard> shardIdToShardMap,
Map<String, Set<String>> shardIdToChildShardIdsMap, Map<String, Set<String>> shardIdToChildShardIdsMap,
List<KinesisClientLease> trackedLeases, List<KinesisClientLease> trackedLeases,
@ -733,7 +684,7 @@ class ShardSyncer {
* @throws InvalidStateException * @throws InvalidStateException
* @throws DependencyException * @throws DependencyException
*/ */
static synchronized void cleanupLeaseForClosedShard(String closedShardId, synchronized void cleanupLeaseForClosedShard(String closedShardId,
Set<String> childShardIds, Set<String> childShardIds,
Map<String, KinesisClientLease> trackedLeases, Map<String, KinesisClientLease> trackedLeases,
ILeaseManager<KinesisClientLease> leaseManager) ILeaseManager<KinesisClientLease> leaseManager)
@ -774,7 +725,7 @@ class ShardSyncer {
* @param shard * @param shard
* @return * @return
*/ */
static KinesisClientLease newKCLLease(Shard shard) { KinesisClientLease newKCLLease(Shard shard) {
KinesisClientLease newLease = new KinesisClientLease(); KinesisClientLease newLease = new KinesisClientLease();
newLease.setLeaseKey(shard.getShardId()); newLease.setLeaseKey(shard.getShardId());
List<String> parentShardIds = new ArrayList<String>(2); List<String> parentShardIds = new ArrayList<String>(2);
@ -796,7 +747,7 @@ class ShardSyncer {
* @param shards List of shards * @param shards List of shards
* @return ShardId->Shard map * @return ShardId->Shard map
*/ */
static Map<String, Shard> constructShardIdToShardMap(List<Shard> shards) { Map<String, Shard> constructShardIdToShardMap(List<Shard> shards) {
Map<String, Shard> shardIdToShardMap = new HashMap<String, Shard>(); Map<String, Shard> shardIdToShardMap = new HashMap<String, Shard>();
for (Shard shard : shards) { for (Shard shard : shards) {
shardIdToShardMap.put(shard.getShardId(), shard); shardIdToShardMap.put(shard.getShardId(), shard);
@ -811,7 +762,7 @@ class ShardSyncer {
* @param allShards All shards returved via DescribeStream. We assume this to represent a consistent shard list. * @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. * @return List of open shards (shards at the tip of the stream) - may include shards that are not yet active.
*/ */
static List<Shard> getOpenShards(List<Shard> allShards) { List<Shard> getOpenShards(List<Shard> allShards) {
List<Shard> openShards = new ArrayList<Shard>(); List<Shard> openShards = new ArrayList<Shard>();
for (Shard shard : allShards) { for (Shard shard : allShards) {
String endingSequenceNumber = shard.getSequenceNumberRange().getEndingSequenceNumber(); String endingSequenceNumber = shard.getSequenceNumberRange().getEndingSequenceNumber();
@ -823,7 +774,7 @@ class ShardSyncer {
return openShards; return openShards;
} }
private static ExtendedSequenceNumber convertToCheckpoint(InitialPositionInStreamExtended position) { private ExtendedSequenceNumber convertToCheckpoint(InitialPositionInStreamExtended position) {
ExtendedSequenceNumber checkpoint = null; ExtendedSequenceNumber checkpoint = null;
if (position.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)) { if (position.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)) {

View file

@ -48,6 +48,7 @@ class ShutdownTask implements ITask {
private final TaskType taskType = TaskType.SHUTDOWN; private final TaskType taskType = TaskType.SHUTDOWN;
private final long backoffTimeMillis; private final long backoffTimeMillis;
private final GetRecordsCache getRecordsCache; private final GetRecordsCache getRecordsCache;
private final ShardSyncer shardSyncer;
/** /**
* Constructor. * Constructor.
@ -63,7 +64,8 @@ class ShutdownTask implements ITask {
boolean ignoreUnexpectedChildShards, boolean ignoreUnexpectedChildShards,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
long backoffTimeMillis, long backoffTimeMillis,
GetRecordsCache getRecordsCache) { GetRecordsCache getRecordsCache,
ShardSyncer shardSyncer) {
this.shardInfo = shardInfo; this.shardInfo = shardInfo;
this.recordProcessor = recordProcessor; this.recordProcessor = recordProcessor;
this.recordProcessorCheckpointer = recordProcessorCheckpointer; this.recordProcessorCheckpointer = recordProcessorCheckpointer;
@ -75,6 +77,7 @@ class ShutdownTask implements ITask {
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
this.backoffTimeMillis = backoffTimeMillis; this.backoffTimeMillis = backoffTimeMillis;
this.getRecordsCache = getRecordsCache; this.getRecordsCache = getRecordsCache;
this.shardSyncer = shardSyncer;
} }
/* /*
@ -127,7 +130,7 @@ class ShutdownTask implements ITask {
if (reason == ShutdownReason.TERMINATE) { if (reason == ShutdownReason.TERMINATE) {
LOG.debug("Looking for child shards of shard " + shardInfo.getShardId()); LOG.debug("Looking for child shards of shard " + shardInfo.getShardId());
// create leases for the child shards // create leases for the child shards
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy,
leaseManager, leaseManager,
initialPositionInStream, initialPositionInStream,
cleanupLeasesOfCompletedShards, cleanupLeasesOfCompletedShards,

View file

@ -33,11 +33,12 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.function.Consumer; import java.util.function.Consumer;
import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import com.amazonaws.AmazonWebServiceClient;
import com.amazonaws.ClientConfiguration; import com.amazonaws.ClientConfiguration;
import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.AWSCredentialsProvider;
import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.client.builder.AwsClientBuilder;
@ -69,7 +70,6 @@ import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.common.util.concurrent.ThreadFactoryBuilder;
import lombok.NonNull;
import lombok.Setter; import lombok.Setter;
import lombok.experimental.Accessors; import lombok.experimental.Accessors;
@ -84,6 +84,8 @@ public class Worker implements Runnable {
private static final int MAX_INITIALIZATION_ATTEMPTS = 20; private static final int MAX_INITIALIZATION_ATTEMPTS = 20;
private static final WorkerStateChangeListener DEFAULT_WORKER_STATE_CHANGE_LISTENER = new NoOpWorkerStateChangeListener(); 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<KinesisClientLease> DEFAULT_LEASE_SELECTOR = new GenericLeaseSelector<KinesisClientLease>();
private WorkerLog wlog = new WorkerLog(); private WorkerLog wlog = new WorkerLog();
@ -114,6 +116,7 @@ public class Worker implements Runnable {
private volatile boolean shutdown; private volatile boolean shutdown;
private volatile long shutdownStartTimeMillis; private volatile long shutdownStartTimeMillis;
private volatile boolean shutdownComplete = false; private volatile boolean shutdownComplete = false;
private final ShardSyncer shardSyncer;
// Holds consumers for shards the worker is currently tracking. Key is shard // Holds consumers for shards the worker is currently tracking. Key is shard
// info, value is ShardConsumer. // info, value is ShardConsumer.
@ -388,6 +391,7 @@ public class Worker implements Runnable {
config.getShardSyncIntervalMillis(), config.shouldCleanupLeasesUponShardCompletion(), null, config.getShardSyncIntervalMillis(), config.shouldCleanupLeasesUponShardCompletion(), null,
new KinesisClientLibLeaseCoordinator( new KinesisClientLibLeaseCoordinator(
new KinesisClientLeaseManager(config.getTableName(), dynamoDBClient), new KinesisClientLeaseManager(config.getTableName(), dynamoDBClient),
DEFAULT_LEASE_SELECTOR,
config.getWorkerIdentifier(), config.getWorkerIdentifier(),
config.getFailoverTimeMillis(), config.getFailoverTimeMillis(),
config.getEpsilonMillis(), config.getEpsilonMillis(),
@ -405,7 +409,8 @@ public class Worker implements Runnable {
config.getShardPrioritizationStrategy(), config.getShardPrioritizationStrategy(),
config.getRetryGetRecordsInSeconds(), config.getRetryGetRecordsInSeconds(),
config.getMaxGetRecordsThreadPool(), config.getMaxGetRecordsThreadPool(),
DEFAULT_WORKER_STATE_CHANGE_LISTENER); DEFAULT_WORKER_STATE_CHANGE_LISTENER,
DEFAULT_LEASE_CLEANUP_VALIDATOR );
// If a region name was explicitly specified, use it as the region for Amazon Kinesis and Amazon DynamoDB. // If a region name was explicitly specified, use it as the region for Amazon Kinesis and Amazon DynamoDB.
if (config.getRegionName() != null) { if (config.getRegionName() != null) {
@ -465,7 +470,7 @@ public class Worker implements Runnable {
this(applicationName, recordProcessorFactory, config, streamConfig, initialPositionInStream, parentShardPollIntervalMillis, this(applicationName, recordProcessorFactory, config, streamConfig, initialPositionInStream, parentShardPollIntervalMillis,
shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService,
metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist,
shardPrioritization, Optional.empty(), Optional.empty(), DEFAULT_WORKER_STATE_CHANGE_LISTENER); shardPrioritization, Optional.empty(), Optional.empty(), DEFAULT_WORKER_STATE_CHANGE_LISTENER, DEFAULT_LEASE_CLEANUP_VALIDATOR );
} }
/** /**
@ -503,6 +508,8 @@ public class Worker implements Runnable {
* Time in seconds to wait before the worker retries to get a record. * Time in seconds to wait before the worker retries to get a record.
* @param maxGetRecordsThreadPool * @param maxGetRecordsThreadPool
* Max number of threads in the getRecords thread pool. * Max number of threads in the getRecords thread pool.
* @param leaseCleanupValidator
* leaseCleanupValidator instance used to validate leases
*/ */
// NOTE: This has package level access solely for testing // NOTE: This has package level access solely for testing
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
@ -512,7 +519,8 @@ public class Worker implements Runnable {
KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization,
Optional<Integer> retryGetRecordsInSeconds, Optional<Integer> maxGetRecordsThreadPool, WorkerStateChangeListener workerStateChangeListener) { Optional<Integer> retryGetRecordsInSeconds, Optional<Integer> maxGetRecordsThreadPool, WorkerStateChangeListener workerStateChangeListener,
LeaseCleanupValidator leaseCleanupValidator) {
this.applicationName = applicationName; this.applicationName = applicationName;
this.recordProcessorFactory = recordProcessorFactory; this.recordProcessorFactory = recordProcessorFactory;
this.config = config; this.config = config;
@ -525,9 +533,10 @@ public class Worker implements Runnable {
this.executorService = execService; this.executorService = execService;
this.leaseCoordinator = leaseCoordinator; this.leaseCoordinator = leaseCoordinator;
this.metricsFactory = metricsFactory; this.metricsFactory = metricsFactory;
this.shardSyncer = new ShardSyncer(leaseCleanupValidator);
this.controlServer = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(), this.controlServer = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(),
initialPositionInStream, cleanupLeasesUponShardCompletion, config.shouldIgnoreUnexpectedChildShards(), initialPositionInStream, cleanupLeasesUponShardCompletion, config.shouldIgnoreUnexpectedChildShards(),
shardSyncIdleTimeMillis, metricsFactory, executorService); shardSyncIdleTimeMillis, metricsFactory, executorService, shardSyncer);
this.taskBackoffTimeMillis = taskBackoffTimeMillis; this.taskBackoffTimeMillis = taskBackoffTimeMillis;
this.failoverTimeMillis = failoverTimeMillis; this.failoverTimeMillis = failoverTimeMillis;
this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist; this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist;
@ -629,7 +638,7 @@ public class Worker implements Runnable {
LOG.info("Syncing Kinesis shard info"); LOG.info("Syncing Kinesis shard info");
ShardSyncTask shardSyncTask = new ShardSyncTask(streamConfig.getStreamProxy(), ShardSyncTask shardSyncTask = new ShardSyncTask(streamConfig.getStreamProxy(),
leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion, leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion,
config.shouldIgnoreUnexpectedChildShards(), 0L); config.shouldIgnoreUnexpectedChildShards(), 0L, shardSyncer);
result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call();
} else { } else {
LOG.info("Skipping shard sync per config setting (and lease table is not empty)"); LOG.info("Skipping shard sync per config setting (and lease table is not empty)");
@ -996,7 +1005,8 @@ public class Worker implements Runnable {
skipShardSyncAtWorkerInitializationIfLeasesExist, skipShardSyncAtWorkerInitializationIfLeasesExist,
retryGetRecordsInSeconds, retryGetRecordsInSeconds,
maxGetRecordsThreadPool, maxGetRecordsThreadPool,
config); config,
shardSyncer);
} }
@ -1158,6 +1168,10 @@ public class Worker implements Runnable {
private IKinesisProxy kinesisProxy; private IKinesisProxy kinesisProxy;
@Setter @Accessors(fluent = true) @Setter @Accessors(fluent = true)
private WorkerStateChangeListener workerStateChangeListener; private WorkerStateChangeListener workerStateChangeListener;
@Setter @Accessors(fluent = true)
private LeaseCleanupValidator leaseCleanupValidator;
@Setter @Accessors(fluent = true)
private LeaseSelector<KinesisClientLease> leaseSelector;
@VisibleForTesting @VisibleForTesting
AmazonKinesis getKinesisClient() { AmazonKinesis getKinesisClient() {
@ -1272,6 +1286,14 @@ public class Worker implements Runnable {
workerStateChangeListener = DEFAULT_WORKER_STATE_CHANGE_LISTENER; workerStateChangeListener = DEFAULT_WORKER_STATE_CHANGE_LISTENER;
} }
if(leaseCleanupValidator == null) {
leaseCleanupValidator = DEFAULT_LEASE_CLEANUP_VALIDATOR;
}
if(leaseSelector == null) {
leaseSelector = DEFAULT_LEASE_SELECTOR;
}
return new Worker(config.getApplicationName(), return new Worker(config.getApplicationName(),
recordProcessorFactory, recordProcessorFactory,
config, config,
@ -1287,6 +1309,7 @@ public class Worker implements Runnable {
config.shouldCleanupLeasesUponShardCompletion(), config.shouldCleanupLeasesUponShardCompletion(),
null, null,
new KinesisClientLibLeaseCoordinator(leaseManager, new KinesisClientLibLeaseCoordinator(leaseManager,
leaseSelector,
config.getWorkerIdentifier(), config.getWorkerIdentifier(),
config.getFailoverTimeMillis(), config.getFailoverTimeMillis(),
config.getEpsilonMillis(), config.getEpsilonMillis(),
@ -1304,7 +1327,8 @@ public class Worker implements Runnable {
shardPrioritization, shardPrioritization,
config.getRetryGetRecordsInSeconds(), config.getRetryGetRecordsInSeconds(),
config.getMaxGetRecordsThreadPool(), config.getMaxGetRecordsThreadPool(),
workerStateChangeListener); workerStateChangeListener,
leaseCleanupValidator);
} }
<R, T extends AwsClientBuilder<T, R>> R createClient(final T builder, <R, T extends AwsClientBuilder<T, R>> R createClient(final T builder,

View file

@ -0,0 +1,43 @@
package com.amazonaws.services.kinesis.leases.impl;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/**
* GenericLeaseSelector abstracts away the lease selection logic from the application code that's using leasing.
* It owns filtering of the leases to be taken.
*/
public class GenericLeaseSelector<T extends Lease> implements LeaseSelector<T> {
/**
* Provides the list of leases to be taken.
* @param expiredLeases list of leases that are currently expired
* @param numLeasesToReachTarget the number of leases to be taken
* @return
*/
@Override
public Set<T> getLeasesToTakeFromExpiredLeases(List<T> expiredLeases, int numLeasesToReachTarget) {
Set<T> leasesToTake = new HashSet<T>();
// If we have expired leases, get up to <needed> leases from expiredLeases
for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) {
leasesToTake.add(expiredLeases.remove(0));
}
return leasesToTake;
}
/**
* Provides the number of leases that should be taken by the worker.
* @param allLeases list of all existing leases
* @return
*/
@Override
public int getLeaseCountThatCanBeTaken(Collection<T> allLeases) {
return allLeases.size();
}
}

View file

@ -26,6 +26,7 @@ import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -82,9 +83,14 @@ public class LeaseCoordinator<T extends Lease> {
private ScheduledExecutorService leaseCoordinatorThreadPool; private ScheduledExecutorService leaseCoordinatorThreadPool;
private final ExecutorService leaseRenewalThreadpool; private final ExecutorService leaseRenewalThreadpool;
private volatile boolean running = false; private volatile boolean running = false;
private ScheduledFuture<?> takerFuture; private ScheduledFuture<?> takerFuture;
private static <T extends Lease> LeaseSelector<T> getDefaultLeaseSelector() {
return new GenericLeaseSelector<>();
}
/** /**
* Constructor. * Constructor.
* *
@ -100,6 +106,23 @@ public class LeaseCoordinator<T extends Lease> {
this(leaseManager, workerIdentifier, leaseDurationMillis, epsilonMillis, new LogMetricsFactory()); this(leaseManager, workerIdentifier, leaseDurationMillis, epsilonMillis, new LogMetricsFactory());
} }
/**
* Constructor.
*
* @param leaseManager LeaseManager instance to use
* @param leaseSelector LeaseSelector instance to use
* @param workerIdentifier Identifies the worker (e.g. useful to track lease ownership)
* @param leaseDurationMillis Duration of a lease
* @param epsilonMillis Allow for some variance when calculating lease expirations
*/
public LeaseCoordinator(ILeaseManager<T> leaseManager,
LeaseSelector<T> leaseSelector,
String workerIdentifier,
long leaseDurationMillis,
long epsilonMillis) {
this(leaseManager, leaseSelector, workerIdentifier, leaseDurationMillis, epsilonMillis, new LogMetricsFactory());
}
/** /**
* Constructor. * Constructor.
* *
@ -119,6 +142,27 @@ public class LeaseCoordinator<T extends Lease> {
KinesisClientLibConfiguration.DEFAULT_MAX_LEASE_RENEWAL_THREADS, metricsFactory); KinesisClientLibConfiguration.DEFAULT_MAX_LEASE_RENEWAL_THREADS, metricsFactory);
} }
/**
* Constructor.
*
* @param leaseManager LeaseManager instance to use
* @param leaseSelector LeaseSelector instance to use
* @param workerIdentifier Identifies the worker (e.g. useful to track lease ownership)
* @param leaseDurationMillis Duration of a lease
* @param epsilonMillis Allow for some variance when calculating lease expirations
* @param metricsFactory Used to publish metrics about lease operations
*/
public LeaseCoordinator(ILeaseManager<T> leaseManager,
LeaseSelector<T> leaseSelector,
String workerIdentifier,
long leaseDurationMillis,
long epsilonMillis,
IMetricsFactory metricsFactory) {
this(leaseManager, leaseSelector, workerIdentifier, leaseDurationMillis, epsilonMillis,
DEFAULT_MAX_LEASES_FOR_WORKER, DEFAULT_MAX_LEASES_TO_STEAL_AT_ONE_TIME,
KinesisClientLibConfiguration.DEFAULT_MAX_LEASE_RENEWAL_THREADS, metricsFactory);
}
/** /**
* Constructor. * Constructor.
* *
@ -138,8 +182,33 @@ public class LeaseCoordinator<T extends Lease> {
int maxLeasesToStealAtOneTime, int maxLeasesToStealAtOneTime,
int maxLeaseRenewerThreadCount, int maxLeaseRenewerThreadCount,
IMetricsFactory metricsFactory) { IMetricsFactory metricsFactory) {
this(leaseManager, getDefaultLeaseSelector(), workerIdentifier, leaseDurationMillis, epsilonMillis,
maxLeasesForWorker, maxLeasesToStealAtOneTime, maxLeaseRenewerThreadCount, metricsFactory);
}
/**
* Constructor.
*
* @param leaseManager LeaseManager instance to use
* @param leaseSelector LeaseSelector instance to use
* @param workerIdentifier Identifies the worker (e.g. useful to track lease ownership)
* @param leaseDurationMillis Duration of a lease
* @param epsilonMillis Allow for some variance when calculating lease expirations
* @param maxLeasesForWorker Max leases this Worker can handle at a time
* @param maxLeasesToStealAtOneTime Steal up to these many leases at a time (for load balancing)
* @param metricsFactory Used to publish metrics about lease operations
*/
public LeaseCoordinator(ILeaseManager<T> leaseManager,
LeaseSelector<T> leaseSelector,
String workerIdentifier,
long leaseDurationMillis,
long epsilonMillis,
int maxLeasesForWorker,
int maxLeasesToStealAtOneTime,
int maxLeaseRenewerThreadCount,
IMetricsFactory metricsFactory) {
this.leaseRenewalThreadpool = getLeaseRenewalExecutorService(maxLeaseRenewerThreadCount); this.leaseRenewalThreadpool = getLeaseRenewalExecutorService(maxLeaseRenewerThreadCount);
this.leaseTaker = new LeaseTaker<T>(leaseManager, workerIdentifier, leaseDurationMillis) this.leaseTaker = new LeaseTaker<T>(leaseManager, leaseSelector, workerIdentifier, leaseDurationMillis)
.withMaxLeasesForWorker(maxLeasesForWorker) .withMaxLeasesForWorker(maxLeasesForWorker)
.withMaxLeasesToStealAtOneTime(maxLeasesToStealAtOneTime); .withMaxLeasesToStealAtOneTime(maxLeasesToStealAtOneTime);
this.leaseRenewer = new LeaseRenewer<T>( this.leaseRenewer = new LeaseRenewer<T>(

View file

@ -26,6 +26,7 @@ import java.util.Set;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -59,6 +60,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
}; };
private final ILeaseManager<T> leaseManager; private final ILeaseManager<T> leaseManager;
private final LeaseSelector<T> leaseSelector;
private final String workerIdentifier; private final String workerIdentifier;
private final Map<String, T> allLeases = new HashMap<String, T>(); private final Map<String, T> allLeases = new HashMap<String, T>();
private final long leaseDurationNanos; private final long leaseDurationNanos;
@ -67,8 +69,18 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
private long lastScanTimeNanos = 0L; private long lastScanTimeNanos = 0L;
private static <T extends Lease> LeaseSelector<T> getDefaultLeaseSelector() {
return new GenericLeaseSelector<>();
}
public LeaseTaker(ILeaseManager<T> leaseManager, String workerIdentifier, long leaseDurationMillis) { public LeaseTaker(ILeaseManager<T> leaseManager, String workerIdentifier, long leaseDurationMillis) {
this(leaseManager, getDefaultLeaseSelector(), workerIdentifier, leaseDurationMillis);
}
public LeaseTaker(ILeaseManager<T> leaseManager, LeaseSelector<T> leaseSelector,
String workerIdentifier, long leaseDurationMillis) {
this.leaseManager = leaseManager; this.leaseManager = leaseManager;
this.leaseSelector = leaseSelector;
this.workerIdentifier = workerIdentifier; this.workerIdentifier = workerIdentifier;
this.leaseDurationNanos = TimeUnit.MILLISECONDS.toNanos(leaseDurationMillis); this.leaseDurationNanos = TimeUnit.MILLISECONDS.toNanos(leaseDurationMillis);
} }
@ -332,7 +344,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
Set<T> leasesToTake = new HashSet<T>(); Set<T> leasesToTake = new HashSet<T>();
IMetricsScope metrics = MetricsHelper.getMetricsScope(); IMetricsScope metrics = MetricsHelper.getMetricsScope();
int numLeases = allLeases.size(); int numLeases = leaseSelector.getLeaseCountThatCanBeTaken(allLeases.values());
int numWorkers = leaseCounts.size(); int numWorkers = leaseCounts.size();
if (numLeases == 0) { if (numLeases == 0) {
@ -382,10 +394,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
int originalExpiredLeasesSize = expiredLeases.size(); int originalExpiredLeasesSize = expiredLeases.size();
if (expiredLeases.size() > 0) { if (expiredLeases.size() > 0) {
// If we have expired leases, get up to <needed> leases from expiredLeases leasesToTake = leaseSelector.getLeasesToTakeFromExpiredLeases(expiredLeases, numLeasesToReachTarget);
for (; numLeasesToReachTarget > 0 && expiredLeases.size() > 0; numLeasesToReachTarget--) {
leasesToTake.add(expiredLeases.remove(0));
}
} else { } else {
// If there are no expired leases and we need a lease, consider stealing. // If there are no expired leases and we need a lease, consider stealing.
List<T> leasesToSteal = chooseLeasesToSteal(leaseCounts, numLeasesToReachTarget, target); List<T> leasesToSteal = chooseLeasesToSteal(leaseCounts, numLeasesToReachTarget, target);

View file

@ -0,0 +1,30 @@
package com.amazonaws.services.kinesis.leases.interfaces;
import com.amazonaws.services.kinesis.leases.impl.Lease;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* LeaseSelector abstracts away the lease selection logic from the application code that's using leasing.
* It owns filtering of the leases to be taken.
*/
public interface LeaseSelector<T extends Lease> {
/**
* Provides the list of leases to be taken.
* @param expiredLeases list of leases that are currently expired
* @param numLeasesToReachTarget the number of leases to be taken
* @return
*/
Set<T> getLeasesToTakeFromExpiredLeases(List<T> expiredLeases, int numLeasesToReachTarget);
/**
* Provides the number of leases that should be taken by the worker.
* @param allLeases list of all existing leases
* @return
*/
int getLeaseCountThatCanBeTaken(Collection<T> allLeases);
}

View file

@ -23,7 +23,8 @@ import java.util.UUID;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.auth.SystemPropertiesCredentialsProvider; import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import junit.framework.Assert; import junit.framework.Assert;
import org.junit.Before; import org.junit.Before;
@ -56,6 +57,7 @@ public class KinesisClientLibLeaseCoordinatorIntegrationTest {
@Before @Before
public void setUp() throws ProvisionedThroughputException, DependencyException, InvalidStateException { public void setUp() throws ProvisionedThroughputException, DependencyException, InvalidStateException {
final boolean useConsistentReads = true; final boolean useConsistentReads = true;
LeaseSelector<KinesisClientLease> leaseSelector = new GenericLeaseSelector<>();
if (leaseManager == null) { if (leaseManager == null) {
AmazonDynamoDBClient ddb = new AmazonDynamoDBClient(new DefaultAWSCredentialsProviderChain()); AmazonDynamoDBClient ddb = new AmazonDynamoDBClient(new DefaultAWSCredentialsProviderChain());
leaseManager = leaseManager =
@ -63,7 +65,7 @@ public class KinesisClientLibLeaseCoordinatorIntegrationTest {
} }
leaseManager.createLeaseTableIfNotExists(10L, 10L); leaseManager.createLeaseTableIfNotExists(10L, 10L);
leaseManager.deleteAll(); leaseManager.deleteAll();
coordinator = new KinesisClientLibLeaseCoordinator(leaseManager, WORKER_ID, 5000L, 50L); coordinator = new KinesisClientLibLeaseCoordinator(leaseManager, WORKER_ID, 5000L, 50L, leaseSelector);
coordinator.start(); coordinator.start();
} }

View file

@ -19,6 +19,9 @@ import static org.mockito.Mockito.doReturn;
import java.util.UUID; import java.util.UUID;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import junit.framework.Assert; import junit.framework.Assert;
import org.junit.Before; import org.junit.Before;
@ -54,7 +57,8 @@ public class KinesisClientLibLeaseCoordinatorTest {
MockitoAnnotations.initMocks(this); MockitoAnnotations.initMocks(this);
// Set up lease coordinator // Set up lease coordinator
doReturn(true).when(mockLeaseManager).createLeaseTableIfNotExists(anyLong(), anyLong()); doReturn(true).when(mockLeaseManager).createLeaseTableIfNotExists(anyLong(), anyLong());
leaseCoordinator = new KinesisClientLibLeaseCoordinator(mockLeaseManager, WORK_ID, TEST_LONG, TEST_LONG); LeaseSelector<KinesisClientLease> leaseSelector = new GenericLeaseSelector<>();
leaseCoordinator = new KinesisClientLibLeaseCoordinator(mockLeaseManager, WORK_ID, TEST_LONG, TEST_LONG, leaseSelector);
} }
@Test(expected = ShutdownException.class) @Test(expected = ShutdownException.class)

View file

@ -97,6 +97,7 @@ public class ShardConsumerTest {
private final boolean skipCheckpointValidationValue = false; private final boolean skipCheckpointValidationValue = false;
private static final InitialPositionInStreamExtended INITIAL_POSITION_LATEST = private static final InitialPositionInStreamExtended INITIAL_POSITION_LATEST =
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST); InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST);
private static final ShardSyncer shardSyncer = new ShardSyncer(new KinesisLeaseCleanupValidator());
// Use Executors.newFixedThreadPool since it returns ThreadPoolExecutor, which is // Use Executors.newFixedThreadPool since it returns ThreadPoolExecutor, which is
// ... a non-final public class, and so can be mocked and spied. // ... a non-final public class, and so can be mocked and spied.
@ -161,7 +162,8 @@ public class ShardConsumerTest {
metricsFactory, metricsFactory,
taskBackoffTimeMillis, taskBackoffTimeMillis,
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
config); config,
shardSyncer);
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS)));
consumer.consumeShard(); // initialize consumer.consumeShard(); // initialize
@ -209,7 +211,8 @@ public class ShardConsumerTest {
metricsFactory, metricsFactory,
taskBackoffTimeMillis, taskBackoffTimeMillis,
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
config); config,
shardSyncer);
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS)));
consumer.consumeShard(); // initialize consumer.consumeShard(); // initialize
@ -251,7 +254,8 @@ public class ShardConsumerTest {
metricsFactory, metricsFactory,
taskBackoffTimeMillis, taskBackoffTimeMillis,
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
config); config,
shardSyncer);
final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123");
final ExtendedSequenceNumber pendingCheckpointSequenceNumber = null; final ExtendedSequenceNumber pendingCheckpointSequenceNumber = null;
@ -370,7 +374,8 @@ public class ShardConsumerTest {
dataFetcher, dataFetcher,
Optional.empty(), Optional.empty(),
Optional.empty(), Optional.empty(),
config); config,
shardSyncer);
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS)));
consumer.consumeShard(); // check on parent shards consumer.consumeShard(); // check on parent shards
@ -514,7 +519,8 @@ public class ShardConsumerTest {
dataFetcher, dataFetcher,
Optional.empty(), Optional.empty(),
Optional.empty(), Optional.empty(),
config); config,
shardSyncer);
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS)));
consumer.consumeShard(); // check on parent shards consumer.consumeShard(); // check on parent shards
@ -651,7 +657,8 @@ public class ShardConsumerTest {
dataFetcher, dataFetcher,
Optional.empty(), Optional.empty(),
Optional.empty(), Optional.empty(),
config); config,
shardSyncer);
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS)));
consumer.consumeShard(); // check on parent shards consumer.consumeShard(); // check on parent shards
@ -721,7 +728,8 @@ public class ShardConsumerTest {
metricsFactory, metricsFactory,
taskBackoffTimeMillis, taskBackoffTimeMillis,
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
config); config,
shardSyncer);
GetRecordsCache getRecordsCache = spy(consumer.getGetRecordsCache()); GetRecordsCache getRecordsCache = spy(consumer.getGetRecordsCache());
@ -774,7 +782,8 @@ public class ShardConsumerTest {
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
Optional.empty(), Optional.empty(),
Optional.empty(), Optional.empty(),
config); config,
shardSyncer);
assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(), assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(),
SynchronousGetRecordsRetrievalStrategy.class); SynchronousGetRecordsRetrievalStrategy.class);
@ -804,7 +813,8 @@ public class ShardConsumerTest {
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
Optional.of(1), Optional.of(1),
Optional.of(2), Optional.of(2),
config); config,
shardSyncer);
assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(), assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(),
AsynchronousGetRecordsRetrievalStrategy.class); AsynchronousGetRecordsRetrievalStrategy.class);
@ -843,7 +853,8 @@ public class ShardConsumerTest {
metricsFactory, metricsFactory,
taskBackoffTimeMillis, taskBackoffTimeMillis,
KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST,
config); config,
shardSyncer);
shardConsumer.consumeShard(); shardConsumer.consumeShard();

View file

@ -52,6 +52,7 @@ public class ShardSyncTaskIntegrationTest {
private static AWSCredentialsProvider credentialsProvider; private static AWSCredentialsProvider credentialsProvider;
private IKinesisClientLeaseManager leaseManager; private IKinesisClientLeaseManager leaseManager;
private IKinesisProxy kinesisProxy; private IKinesisProxy kinesisProxy;
private final ShardSyncer shardSyncer = new ShardSyncer(new KinesisLeaseCleanupValidator());
/** /**
* @throws java.lang.Exception * @throws java.lang.Exception
@ -125,7 +126,8 @@ public class ShardSyncTaskIntegrationTest {
InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST),
false, false,
false, false,
0L); 0L,
shardSyncer);
syncTask.call(); syncTask.call();
List<KinesisClientLease> leases = leaseManager.listLeases(); List<KinesisClientLease> leases = leaseManager.listLeases();
Set<String> leaseKeys = new HashSet<String>(); Set<String> leaseKeys = new HashSet<String>();

View file

@ -70,6 +70,8 @@ public class ShardSyncerTest {
AmazonDynamoDB ddbClient = DynamoDBEmbedded.create().amazonDynamoDB(); AmazonDynamoDB ddbClient = DynamoDBEmbedded.create().amazonDynamoDB();
LeaseManager<KinesisClientLease> leaseManager = new KinesisClientLeaseManager("tempTestTable", ddbClient); LeaseManager<KinesisClientLease> leaseManager = new KinesisClientLeaseManager("tempTestTable", ddbClient);
private static final int EXPONENT = 128; private static final int EXPONENT = 128;
protected static final KinesisLeaseCleanupValidator leaseCleanupValidator = new KinesisLeaseCleanupValidator();
private static final ShardSyncer shardSyncer = new ShardSyncer(leaseCleanupValidator);
/** /**
* Old/Obsolete max value of a sequence number (2^128 -1). * Old/Obsolete max value of a sequence number (2^128 -1).
*/ */
@ -117,7 +119,7 @@ public class ShardSyncerTest {
List<Shard> shards = new ArrayList<Shard>(); List<Shard> shards = new ArrayList<Shard>();
List<KinesisClientLease> leases = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> leases = new ArrayList<KinesisClientLease>();
Assert.assertTrue(ShardSyncer.determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST).isEmpty()); Assert.assertTrue(shardSyncer.determineNewLeasesToCreate(shards, leases, INITIAL_POSITION_LATEST).isEmpty());
} }
/** /**
@ -136,7 +138,7 @@ public class ShardSyncerTest {
shards.add(ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); shards.add(ShardObjectHelper.newShard(shardId1, null, null, sequenceRange));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST);
Assert.assertEquals(2, newLeases.size()); Assert.assertEquals(2, newLeases.size());
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
expectedLeaseShardIds.add(shardId0); expectedLeaseShardIds.add(shardId0);
@ -169,7 +171,7 @@ public class ShardSyncerTest {
inconsistentShardIds.add(shardId2); inconsistentShardIds.add(shardId2);
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds);
Assert.assertEquals(2, newLeases.size()); Assert.assertEquals(2, newLeases.size());
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
expectedLeaseShardIds.add(shardId0); expectedLeaseShardIds.add(shardId0);
@ -227,8 +229,8 @@ public class ShardSyncerTest {
dataFile.deleteOnExit(); dataFile.deleteOnExit();
IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath());
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST,
cleanupLeasesOfCompletedShards); cleanupLeasesOfCompletedShards, false);
List<KinesisClientLease> newLeases = leaseManager.listLeases(); List<KinesisClientLease> newLeases = leaseManager.listLeases();
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
expectedLeaseShardIds.add("shardId-4"); expectedLeaseShardIds.add("shardId-4");
@ -259,8 +261,8 @@ public class ShardSyncerTest {
dataFile.deleteOnExit(); dataFile.deleteOnExit();
IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath());
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_TRIM_HORIZON, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_TRIM_HORIZON,
cleanupLeasesOfCompletedShards); cleanupLeasesOfCompletedShards, false);
List<KinesisClientLease> newLeases = leaseManager.listLeases(); List<KinesisClientLease> newLeases = leaseManager.listLeases();
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
for (int i = 0; i < 11; i++) { for (int i = 0; i < 11; i++) {
@ -290,8 +292,8 @@ public class ShardSyncerTest {
dataFile.deleteOnExit(); dataFile.deleteOnExit();
IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath());
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_AT_TIMESTAMP, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_AT_TIMESTAMP,
cleanupLeasesOfCompletedShards); cleanupLeasesOfCompletedShards, false);
List<KinesisClientLease> newLeases = leaseManager.listLeases(); List<KinesisClientLease> newLeases = leaseManager.listLeases();
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
for (int i = 0; i < 11; i++) { for (int i = 0; i < 11; i++) {
@ -324,8 +326,8 @@ public class ShardSyncerTest {
dataFile.deleteOnExit(); dataFile.deleteOnExit();
IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath());
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_TRIM_HORIZON, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_TRIM_HORIZON,
cleanupLeasesOfCompletedShards); cleanupLeasesOfCompletedShards, false);
dataFile.delete(); dataFile.delete();
} }
@ -349,7 +351,7 @@ public class ShardSyncerTest {
File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1");
dataFile.deleteOnExit(); dataFile.deleteOnExit();
IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath());
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST,
cleanupLeasesOfCompletedShards, true); cleanupLeasesOfCompletedShards, true);
List<KinesisClientLease> newLeases = leaseManager.listLeases(); List<KinesisClientLease> newLeases = leaseManager.listLeases();
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
@ -461,10 +463,11 @@ public class ShardSyncerTest {
// Only need to try two times. // Only need to try two times.
for (int i = 1; i <= 2; i++) { for (int i = 1; i <= 2; i++) {
try { try {
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy,
exceptionThrowingLeaseManager, exceptionThrowingLeaseManager,
position, position,
cleanupLeasesOfCompletedShards); cleanupLeasesOfCompletedShards,
false);
return; return;
} catch (LeasingException e) { } catch (LeasingException e) {
LOG.debug("Catch leasing exception", e); LOG.debug("Catch leasing exception", e);
@ -473,10 +476,11 @@ public class ShardSyncerTest {
exceptionThrowingLeaseManager.clearLeaseManagerThrowingExceptionScenario(); exceptionThrowingLeaseManager.clearLeaseManagerThrowingExceptionScenario();
} }
} else { } else {
ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, shardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy,
leaseManager, leaseManager,
position, position,
cleanupLeasesOfCompletedShards); cleanupLeasesOfCompletedShards,
false);
} }
} }
@ -629,7 +633,7 @@ public class ShardSyncerTest {
throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException, throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException,
KinesisClientLibIOException { KinesisClientLibIOException {
String garbageShardId = "shardId-garbage-001"; String garbageShardId = "shardId-garbage-001";
KinesisClientLease garbageLease = ShardSyncer.newKCLLease(ShardObjectHelper.newShard(garbageShardId, KinesisClientLease garbageLease = shardSyncer.newKCLLease(ShardObjectHelper.newShard(garbageShardId,
null, null,
null, null,
ShardObjectHelper.newSequenceNumberRange("101", null))); ShardObjectHelper.newSequenceNumberRange("101", null)));
@ -654,7 +658,7 @@ public class ShardSyncerTest {
dataFile.deleteOnExit(); dataFile.deleteOnExit();
IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath());
ShardSyncer.bootstrapShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, shardSyncer.bootstrapShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards,
false); false);
List<KinesisClientLease> newLeases = leaseManager.listLeases(); List<KinesisClientLease> newLeases = leaseManager.listLeases();
Assert.assertEquals(2, newLeases.size()); Assert.assertEquals(2, newLeases.size());
@ -690,7 +694,7 @@ public class ShardSyncerTest {
for (InitialPositionInStreamExtended initialPosition : initialPositions) { for (InitialPositionInStreamExtended initialPosition : initialPositions) {
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, initialPosition); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, initialPosition);
Assert.assertEquals(2, newLeases.size()); Assert.assertEquals(2, newLeases.size());
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
expectedLeaseShardIds.add(shardId0); expectedLeaseShardIds.add(shardId0);
@ -722,7 +726,7 @@ public class ShardSyncerTest {
ShardObjectHelper.newSequenceNumberRange("405", null))); ShardObjectHelper.newSequenceNumberRange("405", null)));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST);
Assert.assertEquals(1, newLeases.size()); Assert.assertEquals(1, newLeases.size());
Assert.assertEquals(lastShardId, newLeases.get(0).getLeaseKey()); Assert.assertEquals(lastShardId, newLeases.get(0).getLeaseKey());
} }
@ -747,7 +751,7 @@ public class ShardSyncerTest {
currentLeases.add(newLease("shardId-5")); currentLeases.add(newLease("shardId-5"));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap =
new HashMap<String, ExtendedSequenceNumber>(); new HashMap<String, ExtendedSequenceNumber>();
expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON);
@ -785,7 +789,7 @@ public class ShardSyncerTest {
currentLeases.add(newLease("shardId-7")); currentLeases.add(newLease("shardId-7"));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap =
new HashMap<String, ExtendedSequenceNumber>(); new HashMap<String, ExtendedSequenceNumber>();
expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON);
@ -821,7 +825,7 @@ public class ShardSyncerTest {
currentLeases.add(newLease("shardId-5")); currentLeases.add(newLease("shardId-5"));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap =
new HashMap<String, ExtendedSequenceNumber>(); new HashMap<String, ExtendedSequenceNumber>();
expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON);
@ -861,7 +865,7 @@ public class ShardSyncerTest {
currentLeases.add(newLease("shardId-7")); currentLeases.add(newLease("shardId-7"));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap =
new HashMap<String, ExtendedSequenceNumber>(); new HashMap<String, ExtendedSequenceNumber>();
expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.TRIM_HORIZON);
@ -890,7 +894,7 @@ public class ShardSyncerTest {
List<Shard> shards = constructShardListForGraphB(); List<Shard> shards = constructShardListForGraphB();
List<KinesisClientLease> currentLeases = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> currentLeases = new ArrayList<KinesisClientLease>();
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_TRIM_HORIZON);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap =
new HashMap<String, ExtendedSequenceNumber>(); new HashMap<String, ExtendedSequenceNumber>();
for (int i = 0; i < 11; i++) { for (int i = 0; i < 11; i++) {
@ -927,7 +931,7 @@ public class ShardSyncerTest {
currentLeases.add(newLease("shardId-5")); currentLeases.add(newLease("shardId-5"));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = new HashMap<String, ExtendedSequenceNumber>(); Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = new HashMap<String, ExtendedSequenceNumber>();
expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP);
expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP);
@ -966,7 +970,7 @@ public class ShardSyncerTest {
currentLeases.add(newLease("shardId-7")); currentLeases.add(newLease("shardId-7"));
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = new HashMap<String, ExtendedSequenceNumber>(); Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = new HashMap<String, ExtendedSequenceNumber>();
expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-8", ExtendedSequenceNumber.AT_TIMESTAMP);
expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP); expectedShardIdCheckpointMap.put("shardId-9", ExtendedSequenceNumber.AT_TIMESTAMP);
@ -993,7 +997,7 @@ public class ShardSyncerTest {
List<Shard> shards = constructShardListForGraphB(); List<Shard> shards = constructShardListForGraphB();
List<KinesisClientLease> currentLeases = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> currentLeases = new ArrayList<KinesisClientLease>();
List<KinesisClientLease> newLeases = List<KinesisClientLease> newLeases =
ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP); shardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_AT_TIMESTAMP);
Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap = Map<String, ExtendedSequenceNumber> expectedShardIdCheckpointMap =
new HashMap<String, ExtendedSequenceNumber>(); new HashMap<String, ExtendedSequenceNumber>();
for (int i = 0; i < shards.size(); i++) { for (int i = 0; i < shards.size(); i++) {
@ -1102,7 +1106,7 @@ public class ShardSyncerTest {
@Test @Test
public final void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() { public final void testCheckIfDescendantAndAddNewLeasesForAncestorsNullShardId() {
Map<String, Boolean> memoizationContext = new HashMap<>(); Map<String, Boolean> memoizationContext = new HashMap<>();
Assert.assertFalse(ShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST, Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(null, INITIAL_POSITION_LATEST,
null, null,
null, null,
null, null,
@ -1117,7 +1121,7 @@ public class ShardSyncerTest {
String shardId = "shardId-trimmed"; String shardId = "shardId-trimmed";
Map<String, Shard> kinesisShards = new HashMap<String, Shard>(); Map<String, Shard> kinesisShards = new HashMap<String, Shard>();
Map<String, Boolean> memoizationContext = new HashMap<>(); Map<String, Boolean> memoizationContext = new HashMap<>();
Assert.assertFalse(ShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST,
null, null,
kinesisShards, kinesisShards,
null, null,
@ -1136,7 +1140,7 @@ public class ShardSyncerTest {
shardIdsOfCurrentLeases.add(shardId); shardIdsOfCurrentLeases.add(shardId);
Map<String, KinesisClientLease> newLeaseMap = new HashMap<String, KinesisClientLease>(); Map<String, KinesisClientLease> newLeaseMap = new HashMap<String, KinesisClientLease>();
Map<String, Boolean> memoizationContext = new HashMap<>(); Map<String, Boolean> memoizationContext = new HashMap<>();
Assert.assertTrue(ShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, Assert.assertTrue(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST,
shardIdsOfCurrentLeases, shardIdsOfCurrentLeases,
kinesisShards, kinesisShards,
newLeaseMap, newLeaseMap,
@ -1163,7 +1167,7 @@ public class ShardSyncerTest {
kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null)); kinesisShards.put(shardId, ShardObjectHelper.newShard(shardId, parentShardId, adjacentParentShardId, null));
Map<String, Boolean> memoizationContext = new HashMap<>(); Map<String, Boolean> memoizationContext = new HashMap<>();
Assert.assertFalse(ShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, Assert.assertFalse(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST,
shardIdsOfCurrentLeases, shardIdsOfCurrentLeases,
kinesisShards, kinesisShards,
newLeaseMap, newLeaseMap,
@ -1192,7 +1196,7 @@ public class ShardSyncerTest {
kinesisShards.put(shardId, shard); kinesisShards.put(shardId, shard);
Map<String, Boolean> memoizationContext = new HashMap<>(); Map<String, Boolean> memoizationContext = new HashMap<>();
Assert.assertTrue(ShardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST, Assert.assertTrue(shardSyncer.checkIfDescendantAndAddNewLeasesForAncestors(shardId, INITIAL_POSITION_LATEST,
shardIdsOfCurrentLeases, shardIdsOfCurrentLeases,
kinesisShards, kinesisShards,
newLeaseMap, newLeaseMap,
@ -1209,7 +1213,7 @@ public class ShardSyncerTest {
@Test @Test
public final void testGetParentShardIdsNoParents() { public final void testGetParentShardIdsNoParents() {
Shard shard = new Shard(); Shard shard = new Shard();
Assert.assertTrue(ShardSyncer.getParentShardIds(shard, null).isEmpty()); Assert.assertTrue(shardSyncer.getParentShardIds(shard, null).isEmpty());
} }
/** /**
@ -1219,7 +1223,7 @@ public class ShardSyncerTest {
public final void testGetParentShardIdsTrimmedParents() { public final void testGetParentShardIdsTrimmedParents() {
Map<String, Shard> shardMap = new HashMap<String, Shard>(); Map<String, Shard> shardMap = new HashMap<String, Shard>();
Shard shard = ShardObjectHelper.newShard("shardId-test", "foo", "bar", null); Shard shard = ShardObjectHelper.newShard("shardId-test", "foo", "bar", null);
Assert.assertTrue(ShardSyncer.getParentShardIds(shard, shardMap).isEmpty()); Assert.assertTrue(shardSyncer.getParentShardIds(shard, shardMap).isEmpty());
} }
/** /**
@ -1233,16 +1237,16 @@ public class ShardSyncerTest {
shardMap.put(parentShardId, ShardObjectHelper.newShard(parentShardId, null, null, null)); shardMap.put(parentShardId, ShardObjectHelper.newShard(parentShardId, null, null, null));
Shard shard = ShardObjectHelper.newShard("shardId-test", parentShardId, null, null); Shard shard = ShardObjectHelper.newShard("shardId-test", parentShardId, null, null);
Set<String> parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); Set<String> parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertEquals(1, parentShardIds.size()); Assert.assertEquals(1, parentShardIds.size());
Assert.assertTrue(parentShardIds.contains(parentShardId)); Assert.assertTrue(parentShardIds.contains(parentShardId));
shard.setParentShardId(null); shard.setParentShardId(null);
parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertTrue(parentShardIds.isEmpty()); Assert.assertTrue(parentShardIds.isEmpty());
shard.setAdjacentParentShardId(parentShardId); shard.setAdjacentParentShardId(parentShardId);
parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertEquals(1, parentShardIds.size()); Assert.assertEquals(1, parentShardIds.size());
Assert.assertTrue(parentShardIds.contains(parentShardId)); Assert.assertTrue(parentShardIds.contains(parentShardId));
} }
@ -1263,16 +1267,16 @@ public class ShardSyncerTest {
Shard shard = ShardObjectHelper.newShard("shardId-test", parentShardId, adjacentParentShardId, null); Shard shard = ShardObjectHelper.newShard("shardId-test", parentShardId, adjacentParentShardId, null);
shardMap.put(parentShardId, parent); shardMap.put(parentShardId, parent);
Set<String> parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); Set<String> parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertEquals(1, parentShardIds.size()); Assert.assertEquals(1, parentShardIds.size());
Assert.assertTrue(parentShardIds.contains(parentShardId)); Assert.assertTrue(parentShardIds.contains(parentShardId));
shardMap.remove(parentShardId); shardMap.remove(parentShardId);
parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertTrue(parentShardIds.isEmpty()); Assert.assertTrue(parentShardIds.isEmpty());
shardMap.put(adjacentParentShardId, adjacentParent); shardMap.put(adjacentParentShardId, adjacentParent);
parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertEquals(1, parentShardIds.size()); Assert.assertEquals(1, parentShardIds.size());
Assert.assertTrue(parentShardIds.contains(adjacentParentShardId)); Assert.assertTrue(parentShardIds.contains(adjacentParentShardId));
} }
@ -1292,7 +1296,7 @@ public class ShardSyncerTest {
Shard shard = ShardObjectHelper.newShard("shardId-test", parentShardId, adjacentParentShardId, null); Shard shard = ShardObjectHelper.newShard("shardId-test", parentShardId, adjacentParentShardId, null);
Set<String> parentShardIds = ShardSyncer.getParentShardIds(shard, shardMap); Set<String> parentShardIds = shardSyncer.getParentShardIds(shard, shardMap);
Assert.assertEquals(2, parentShardIds.size()); Assert.assertEquals(2, parentShardIds.size());
Assert.assertTrue(parentShardIds.contains(parentShardId)); Assert.assertTrue(parentShardIds.contains(parentShardId));
Assert.assertTrue(parentShardIds.contains(adjacentParentShardId)); Assert.assertTrue(parentShardIds.contains(adjacentParentShardId));
@ -1310,7 +1314,7 @@ public class ShardSyncerTest {
shard.setParentShardId(parentShardId); shard.setParentShardId(parentShardId);
shard.setAdjacentParentShardId(adjacentParentShardId); shard.setAdjacentParentShardId(adjacentParentShardId);
KinesisClientLease lease = ShardSyncer.newKCLLease(shard); KinesisClientLease lease = shardSyncer.newKCLLease(shard);
Assert.assertEquals(shardId, lease.getLeaseKey()); Assert.assertEquals(shardId, lease.getLeaseKey());
Assert.assertNull(lease.getCheckpoint()); Assert.assertNull(lease.getCheckpoint());
Set<String> parentIds = lease.getParentShardIds(); Set<String> parentIds = lease.getParentShardIds();
@ -1330,7 +1334,7 @@ public class ShardSyncerTest {
shards.add(ShardObjectHelper.newShard("shardId-0", null, null, null)); shards.add(ShardObjectHelper.newShard("shardId-0", null, null, null));
shards.add(ShardObjectHelper.newShard("shardId-1", null, null, null)); shards.add(ShardObjectHelper.newShard("shardId-1", null, null, null));
Map<String, Shard> shardIdToShardMap = ShardSyncer.constructShardIdToShardMap(shards); Map<String, Shard> shardIdToShardMap = shardSyncer.constructShardIdToShardMap(shards);
Assert.assertEquals(shards.size(), shardIdToShardMap.size()); Assert.assertEquals(shards.size(), shardIdToShardMap.size());
for (Shard shard : shards) { for (Shard shard : shards) {
Assert.assertSame(shard, shardIdToShardMap.get(shard.getShardId())); Assert.assertSame(shard, shardIdToShardMap.get(shard.getShardId()));
@ -1347,7 +1351,7 @@ public class ShardSyncerTest {
null, null,
null, null,
ShardObjectHelper.newSequenceNumberRange("123", "345"))); ShardObjectHelper.newSequenceNumberRange("123", "345")));
Assert.assertTrue(ShardSyncer.getOpenShards(shards).isEmpty()); Assert.assertTrue(shardSyncer.getOpenShards(shards).isEmpty());
} }
/** /**
@ -1361,18 +1365,18 @@ public class ShardSyncerTest {
shards.add(ShardObjectHelper.newShard(shardId, null, null, sequenceNumberRange)); shards.add(ShardObjectHelper.newShard(shardId, null, null, sequenceNumberRange));
// Verify shard is considered open when it has a null end sequence number // Verify shard is considered open when it has a null end sequence number
List<Shard> openShards = ShardSyncer.getOpenShards(shards); List<Shard> openShards = shardSyncer.getOpenShards(shards);
Assert.assertEquals(1, openShards.size()); Assert.assertEquals(1, openShards.size());
Assert.assertEquals(shardId, openShards.get(0).getShardId()); Assert.assertEquals(shardId, openShards.get(0).getShardId());
// Close shard before testing for max sequence number // Close shard before testing for max sequence number
sequenceNumberRange.setEndingSequenceNumber("1000"); sequenceNumberRange.setEndingSequenceNumber("1000");
openShards = ShardSyncer.getOpenShards(shards); openShards = shardSyncer.getOpenShards(shards);
Assert.assertTrue(openShards.isEmpty()); Assert.assertTrue(openShards.isEmpty());
// Verify shard is considered closed when the end sequence number is set to max allowed sequence number // Verify shard is considered closed when the end sequence number is set to max allowed sequence number
sequenceNumberRange.setEndingSequenceNumber(MAX_SEQUENCE_NUMBER.toString()); sequenceNumberRange.setEndingSequenceNumber(MAX_SEQUENCE_NUMBER.toString());
openShards = ShardSyncer.getOpenShards(shards); openShards = shardSyncer.getOpenShards(shards);
Assert.assertEquals(0, openShards.size()); Assert.assertEquals(0, openShards.size());
} }
@ -1394,23 +1398,23 @@ public class ShardSyncerTest {
Set<String> currentKinesisShardIds = new HashSet<>(); Set<String> currentKinesisShardIds = new HashSet<>();
currentKinesisShardIds.add(shardId); currentKinesisShardIds.add(shardId);
Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
currentKinesisShardIds.clear(); currentKinesisShardIds.clear();
Assert.assertTrue(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); Assert.assertTrue(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
currentKinesisShardIds.add(parentShardId); currentKinesisShardIds.add(parentShardId);
// Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); // Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
currentKinesisShardIds.clear(); currentKinesisShardIds.clear();
Assert.assertTrue(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); Assert.assertTrue(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
currentKinesisShardIds.add(adjacentParentShardId); currentKinesisShardIds.add(adjacentParentShardId);
// Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); // Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
currentKinesisShardIds.add(parentShardId); currentKinesisShardIds.add(parentShardId);
// Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); // Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
currentKinesisShardIds.add(shardId); currentKinesisShardIds.add(shardId);
Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
} }
/** /**
@ -1431,7 +1435,7 @@ public class ShardSyncerTest {
Set<String> currentKinesisShardIds = new HashSet<>(); Set<String> currentKinesisShardIds = new HashSet<>();
currentKinesisShardIds.add(parentShardId); currentKinesisShardIds.add(parentShardId);
Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
} }
/** /**
@ -1452,7 +1456,7 @@ public class ShardSyncerTest {
Set<String> currentKinesisShardIds = new HashSet<>(); Set<String> currentKinesisShardIds = new HashSet<>();
currentKinesisShardIds.add(adjacentParentShardId); currentKinesisShardIds.add(adjacentParentShardId);
Assert.assertFalse(ShardSyncer.isCandidateForCleanup(lease, currentKinesisShardIds)); Assert.assertFalse(leaseCleanupValidator.isCandidateForCleanup(lease, currentKinesisShardIds));
} }
/** /**
@ -1482,22 +1486,22 @@ public class ShardSyncerTest {
KinesisClientLease childLease2 = newLease(childShardId2); KinesisClientLease childLease2 = newLease(childShardId2);
childLease2.setParentShardIds(parentShardIds); childLease2.setParentShardIds(parentShardIds);
childLease2.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); childLease2.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
Map<String, KinesisClientLease> trackedLeaseMap = ShardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); Map<String, KinesisClientLease> trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases);
// empty list of leases // empty list of leases
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNotNull(leaseManager.getLease(closedShardId)); Assert.assertNotNull(leaseManager.getLease(closedShardId));
// closed shard has not been fully processed yet (checkpoint != SHARD_END) // closed shard has not been fully processed yet (checkpoint != SHARD_END)
trackedLeases.add(leaseForClosedShard); trackedLeases.add(leaseForClosedShard);
trackedLeaseMap = ShardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases);
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNotNull(leaseManager.getLease(closedShardId)); Assert.assertNotNull(leaseManager.getLease(closedShardId));
// closed shard has been fully processed yet (checkpoint == SHARD_END) // closed shard has been fully processed yet (checkpoint == SHARD_END)
leaseForClosedShard.setCheckpoint(ExtendedSequenceNumber.SHARD_END); leaseForClosedShard.setCheckpoint(ExtendedSequenceNumber.SHARD_END);
leaseManager.updateLease(leaseForClosedShard); leaseManager.updateLease(leaseForClosedShard);
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNull(leaseManager.getLease(closedShardId)); Assert.assertNull(leaseManager.getLease(closedShardId));
// lease for only one child exists // lease for only one child exists
@ -1506,27 +1510,27 @@ public class ShardSyncerTest {
leaseManager.createLeaseIfNotExists(leaseForClosedShard); leaseManager.createLeaseIfNotExists(leaseForClosedShard);
leaseManager.createLeaseIfNotExists(childLease1); leaseManager.createLeaseIfNotExists(childLease1);
trackedLeases.add(childLease1); trackedLeases.add(childLease1);
trackedLeaseMap = ShardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases);
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNotNull(leaseManager.getLease(closedShardId)); Assert.assertNotNull(leaseManager.getLease(closedShardId));
// leases for both children exists, but they are both at TRIM_HORIZON // leases for both children exists, but they are both at TRIM_HORIZON
leaseManager.createLeaseIfNotExists(childLease2); leaseManager.createLeaseIfNotExists(childLease2);
trackedLeases.add(childLease2); trackedLeases.add(childLease2);
trackedLeaseMap = ShardSyncer.constructShardIdToKCLLeaseMap(trackedLeases); trackedLeaseMap = shardSyncer.constructShardIdToKCLLeaseMap(trackedLeases);
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNotNull(leaseManager.getLease(closedShardId)); Assert.assertNotNull(leaseManager.getLease(closedShardId));
// leases for both children exists, one is at TRIM_HORIZON // leases for both children exists, one is at TRIM_HORIZON
childLease1.setCheckpoint(new ExtendedSequenceNumber("34890")); childLease1.setCheckpoint(new ExtendedSequenceNumber("34890"));
leaseManager.updateLease(childLease1); leaseManager.updateLease(childLease1);
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNotNull(leaseManager.getLease(closedShardId)); Assert.assertNotNull(leaseManager.getLease(closedShardId));
// leases for both children exists, NONE of them are at TRIM_HORIZON // leases for both children exists, NONE of them are at TRIM_HORIZON
childLease2.setCheckpoint(new ExtendedSequenceNumber("43789")); childLease2.setCheckpoint(new ExtendedSequenceNumber("43789"));
leaseManager.updateLease(childLease2); leaseManager.updateLease(childLease2);
ShardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); shardSyncer.cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
Assert.assertNull(leaseManager.getLease(closedShardId)); Assert.assertNull(leaseManager.getLease(closedShardId));
} }
@ -1546,32 +1550,32 @@ public class ShardSyncerTest {
SequenceNumberRange childSequenceNumberRange = ShardObjectHelper.newSequenceNumberRange("206", "300"); SequenceNumberRange childSequenceNumberRange = ShardObjectHelper.newSequenceNumberRange("206", "300");
Shard child1 = Shard child1 =
ShardObjectHelper.newShard("shardId-54879", expectedClosedShardId, null, childSequenceNumberRange); ShardObjectHelper.newShard("shardId-54879", expectedClosedShardId, null, childSequenceNumberRange);
Map<String, Shard> shardIdToShardMap = ShardSyncer.constructShardIdToShardMap(shards); Map<String, Shard> shardIdToShardMap = shardSyncer.constructShardIdToShardMap(shards);
Map<String, Set<String>> shardIdToChildShardIdsMap = Map<String, Set<String>> shardIdToChildShardIdsMap =
ShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); shardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap);
Set<String> closedShardIds = new HashSet<>(); Set<String> closedShardIds = new HashSet<>();
closedShardIds.add(expectedClosedShardId); closedShardIds.add(expectedClosedShardId);
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
// test for case where shard has been trimmed (absent from list) // test for case where shard has been trimmed (absent from list)
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
// Populate shards. // Populate shards.
shards.add(closedShard); shards.add(closedShard);
shards.add(child1); shards.add(child1);
shardIdToShardMap.put(expectedClosedShardId, closedShard); shardIdToShardMap.put(expectedClosedShardId, closedShard);
shardIdToShardMap.put(child1.getShardId(), child1); shardIdToShardMap.put(child1.getShardId(), child1);
shardIdToChildShardIdsMap = ShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); shardIdToChildShardIdsMap = shardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap);
// test degenerate split/merge // test degenerate split/merge
child1.setHashKeyRange(hashKeyRange); child1.setHashKeyRange(hashKeyRange);
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
// test merge // test merge
child1.setHashKeyRange(ShardObjectHelper.newHashKeyRange("10", "2985")); child1.setHashKeyRange(ShardObjectHelper.newHashKeyRange("10", "2985"));
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
child1.setHashKeyRange(ShardObjectHelper.newHashKeyRange("3", "25")); child1.setHashKeyRange(ShardObjectHelper.newHashKeyRange("3", "25"));
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
// test split // test split
HashKeyRange childHashKeyRange1 = ShardObjectHelper.newHashKeyRange("10", "15"); HashKeyRange childHashKeyRange1 = ShardObjectHelper.newHashKeyRange("10", "15");
@ -1584,8 +1588,8 @@ public class ShardSyncerTest {
childHashKeyRange2); childHashKeyRange2);
shards.add(child2); shards.add(child2);
shardIdToShardMap.put(child2.getShardId(), child2); shardIdToShardMap.put(child2.getShardId(), child2);
shardIdToChildShardIdsMap = ShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); shardIdToChildShardIdsMap = shardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap);
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
} }
/** /**
@ -1602,12 +1606,12 @@ public class ShardSyncerTest {
Shard openShard = Shard openShard =
ShardObjectHelper.newShard(expectedClosedShardId, null, null, sequenceNumberRange, hashKeyRange); ShardObjectHelper.newShard(expectedClosedShardId, null, null, sequenceNumberRange, hashKeyRange);
shards.add(openShard); shards.add(openShard);
Map<String, Shard> shardIdToShardMap = ShardSyncer.constructShardIdToShardMap(shards); Map<String, Shard> shardIdToShardMap = shardSyncer.constructShardIdToShardMap(shards);
Map<String, Set<String>> shardIdToChildShardIdsMap = Map<String, Set<String>> shardIdToChildShardIdsMap =
ShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); shardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap);
Set<String> closedShardIds = new HashSet<>(); Set<String> closedShardIds = new HashSet<>();
closedShardIds.add(expectedClosedShardId); closedShardIds.add(expectedClosedShardId);
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
} }
/** /**
@ -1624,12 +1628,12 @@ public class ShardSyncerTest {
Shard closedShard = Shard closedShard =
ShardObjectHelper.newShard(expectedClosedShardId, null, null, sequenceNumberRange, hashKeyRange); ShardObjectHelper.newShard(expectedClosedShardId, null, null, sequenceNumberRange, hashKeyRange);
shards.add(closedShard); shards.add(closedShard);
Map<String, Shard> shardIdToShardMap = ShardSyncer.constructShardIdToShardMap(shards); Map<String, Shard> shardIdToShardMap = shardSyncer.constructShardIdToShardMap(shards);
Map<String, Set<String>> shardIdToChildShardIdsMap = Map<String, Set<String>> shardIdToChildShardIdsMap =
ShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); shardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap);
Set<String> closedShardIds = new HashSet<>(); Set<String> closedShardIds = new HashSet<>();
closedShardIds.add(expectedClosedShardId); closedShardIds.add(expectedClosedShardId);
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
} }
/** /**
@ -1683,12 +1687,12 @@ public class ShardSyncerTest {
child2HashKeyRange); child2HashKeyRange);
shards.add(child2); shards.add(child2);
Map<String, Shard> shardIdToShardMap = ShardSyncer.constructShardIdToShardMap(shards); Map<String, Shard> shardIdToShardMap = shardSyncer.constructShardIdToShardMap(shards);
Map<String, Set<String>> shardIdToChildShardIdsMap = Map<String, Set<String>> shardIdToChildShardIdsMap =
ShardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap); shardSyncer.constructShardIdToChildShardIdsMap(shardIdToShardMap);
Set<String> closedShardIds = new HashSet<>(); Set<String> closedShardIds = new HashSet<>();
closedShardIds.add(expectedClosedShardId); closedShardIds.add(expectedClosedShardId);
ShardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds); shardSyncer.assertClosedShardsAreCoveredOrAbsent(shardIdToShardMap, shardIdToChildShardIdsMap, closedShardIds);
} }
/** /**

View file

@ -57,6 +57,7 @@ public class ShutdownTaskTest {
defaultParentShardIds, defaultParentShardIds,
ExtendedSequenceNumber.LATEST); ExtendedSequenceNumber.LATEST);
IRecordProcessor defaultRecordProcessor = new TestStreamlet(); IRecordProcessor defaultRecordProcessor = new TestStreamlet();
ShardSyncer shardSyncer = new ShardSyncer(new KinesisLeaseCleanupValidator());
@Mock @Mock
private GetRecordsCache getRecordsCache; private GetRecordsCache getRecordsCache;
@ -111,7 +112,8 @@ public class ShutdownTaskTest {
ignoreUnexpectedChildShards, ignoreUnexpectedChildShards,
leaseManager, leaseManager,
TASK_BACKOFF_TIME_MILLIS, TASK_BACKOFF_TIME_MILLIS,
getRecordsCache); getRecordsCache,
shardSyncer);
TaskResult result = task.call(); TaskResult result = task.call();
Assert.assertNotNull(result.getException()); Assert.assertNotNull(result.getException());
Assert.assertTrue(result.getException() instanceof IllegalArgumentException); Assert.assertTrue(result.getException() instanceof IllegalArgumentException);
@ -139,7 +141,8 @@ public class ShutdownTaskTest {
ignoreUnexpectedChildShards, ignoreUnexpectedChildShards,
leaseManager, leaseManager,
TASK_BACKOFF_TIME_MILLIS, TASK_BACKOFF_TIME_MILLIS,
getRecordsCache); getRecordsCache,
shardSyncer);
TaskResult result = task.call(); TaskResult result = task.call();
Assert.assertNotNull(result.getException()); Assert.assertNotNull(result.getException());
Assert.assertTrue(result.getException() instanceof KinesisClientLibIOException); Assert.assertTrue(result.getException() instanceof KinesisClientLibIOException);
@ -151,7 +154,7 @@ public class ShutdownTaskTest {
*/ */
@Test @Test
public final void testGetTaskType() { public final void testGetTaskType() {
ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, null, 0, getRecordsCache); ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, null, 0, getRecordsCache, shardSyncer);
Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType());
} }

View file

@ -66,6 +66,12 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import com.amazonaws.services.kinesis.leases.impl.GenericLeaseSelector;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
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 org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.hamcrest.Condition; import org.hamcrest.Condition;
@ -111,10 +117,6 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber
import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput; import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput;
import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput;
import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput;
import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease;
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.ILeaseManager; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager;
import com.amazonaws.services.kinesis.metrics.impl.CWMetricsFactory; import com.amazonaws.services.kinesis.metrics.impl.CWMetricsFactory;
import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory;
@ -159,6 +161,7 @@ public class WorkerTest {
private RecordsFetcherFactory recordsFetcherFactory; private RecordsFetcherFactory recordsFetcherFactory;
private KinesisClientLibConfiguration config; private KinesisClientLibConfiguration config;
private ShardSyncer shardSyncer = new ShardSyncer(new KinesisLeaseCleanupValidator());
@Mock @Mock
private KinesisClientLibLeaseCoordinator leaseCoordinator; private KinesisClientLibLeaseCoordinator leaseCoordinator;
@ -503,7 +506,7 @@ public class WorkerTest {
final int numberOfRecordsPerShard = 10; final int numberOfRecordsPerShard = 10;
List<Shard> shardList = createShardListWithOneSplit(); List<Shard> shardList = createShardListWithOneSplit();
List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>();
KinesisClientLease lease = ShardSyncer.newKCLLease(shardList.get(0)); KinesisClientLease lease = shardSyncer.newKCLLease(shardList.get(0));
lease.setCheckpoint(new ExtendedSequenceNumber("2")); lease.setCheckpoint(new ExtendedSequenceNumber("2"));
initialLeases.add(lease); initialLeases.add(lease);
runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config); runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config);
@ -519,7 +522,7 @@ public class WorkerTest {
final int numberOfRecordsPerShard = 10; final int numberOfRecordsPerShard = 10;
List<Shard> shardList = createShardListWithOneSplit(); List<Shard> shardList = createShardListWithOneSplit();
List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>();
KinesisClientLease lease = ShardSyncer.newKCLLease(shardList.get(0)); KinesisClientLease lease = shardSyncer.newKCLLease(shardList.get(0));
lease.setCheckpoint(new ExtendedSequenceNumber("2")); lease.setCheckpoint(new ExtendedSequenceNumber("2"));
initialLeases.add(lease); initialLeases.add(lease);
boolean callProcessRecordsForEmptyRecordList = true; boolean callProcessRecordsForEmptyRecordList = true;
@ -611,7 +614,7 @@ public class WorkerTest {
final List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>(); final List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>();
for (Shard shard : shardList) { for (Shard shard : shardList) {
KinesisClientLease lease = ShardSyncer.newKCLLease(shard); KinesisClientLease lease = shardSyncer.newKCLLease(shard);
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
initialLeases.add(lease); initialLeases.add(lease);
} }
@ -687,7 +690,7 @@ public class WorkerTest {
final List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>(); final List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>();
for (Shard shard : shardList) { for (Shard shard : shardList) {
KinesisClientLease lease = ShardSyncer.newKCLLease(shard); KinesisClientLease lease = shardSyncer.newKCLLease(shard);
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
initialLeases.add(lease); initialLeases.add(lease);
} }
@ -2013,7 +2016,7 @@ public class WorkerTest {
Assert.assertEquals(numShards, shardList.size()); Assert.assertEquals(numShards, shardList.size());
List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> initialLeases = new ArrayList<KinesisClientLease>();
for (Shard shard : shardList) { for (Shard shard : shardList) {
KinesisClientLease lease = ShardSyncer.newKCLLease(shard); KinesisClientLease lease = shardSyncer.newKCLLease(shard);
lease.setCheckpoint(ExtendedSequenceNumber.AT_TIMESTAMP); lease.setCheckpoint(ExtendedSequenceNumber.AT_TIMESTAMP);
initialLeases.add(lease); initialLeases.add(lease);
} }
@ -2077,8 +2080,10 @@ public class WorkerTest {
leaseManager.createLeaseIfNotExists(initialLease); leaseManager.createLeaseIfNotExists(initialLease);
} }
LeaseSelector<KinesisClientLease> leaseSelector = new GenericLeaseSelector<>();
KinesisClientLibLeaseCoordinator leaseCoordinator = KinesisClientLibLeaseCoordinator leaseCoordinator =
new KinesisClientLibLeaseCoordinator(leaseManager, new KinesisClientLibLeaseCoordinator(leaseManager,
leaseSelector,
stageName, stageName,
leaseDurationMillis, leaseDurationMillis,
epsilonMillis, epsilonMillis,

View file

@ -28,6 +28,7 @@ import java.util.Map;
import javax.swing.*; import javax.swing.*;
import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -69,12 +70,14 @@ public class LeaseCoordinatorExerciser {
} }
CWMetricsFactory metricsFactory = new CWMetricsFactory(creds, "testNamespace", 30 * 1000, 1000); CWMetricsFactory metricsFactory = new CWMetricsFactory(creds, "testNamespace", 30 * 1000, 1000);
LeaseSelector<KinesisClientLease> leaseSelector = new GenericLeaseSelector<KinesisClientLease>();
final List<LeaseCoordinator<KinesisClientLease>> coordinators = final List<LeaseCoordinator<KinesisClientLease>> coordinators =
new ArrayList<LeaseCoordinator<KinesisClientLease>>(); new ArrayList<LeaseCoordinator<KinesisClientLease>>();
for (int i = 0; i < numCoordinators; i++) { for (int i = 0; i < numCoordinators; i++) {
String workerIdentifier = "worker-" + Integer.toString(i); String workerIdentifier = "worker-" + Integer.toString(i);
LeaseCoordinator<KinesisClientLease> coord = new LeaseCoordinator<KinesisClientLease>(leaseManager, LeaseCoordinator<KinesisClientLease> coord = new LeaseCoordinator<KinesisClientLease>(leaseManager,
leaseSelector,
workerIdentifier, workerIdentifier,
leaseDurationMillis, leaseDurationMillis,
epsilonMillis, epsilonMillis,

View file

@ -16,6 +16,7 @@ package com.amazonaws.services.kinesis.leases.impl;
import java.util.Map; import java.util.Map;
import com.amazonaws.services.kinesis.leases.interfaces.LeaseSelector;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -26,10 +27,11 @@ public class LeaseTakerIntegrationTest extends LeaseIntegrationTest {
private static final long LEASE_DURATION_MILLIS = 1000L; private static final long LEASE_DURATION_MILLIS = 1000L;
private LeaseTaker<KinesisClientLease> taker; private LeaseTaker<KinesisClientLease> taker;
private static final LeaseSelector<KinesisClientLease> leaseSelector = new GenericLeaseSelector<>();
@Before @Before
public void setUp() { public void setUp() {
taker = new LeaseTaker<KinesisClientLease>(leaseManager, "foo", LEASE_DURATION_MILLIS); taker = new LeaseTaker<KinesisClientLease>(leaseManager, leaseSelector,"foo", LEASE_DURATION_MILLIS);
} }
@Test @Test