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

@ -19,7 +19,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
* and state transitions is contained within the {@link ConsumerState} objects. * and state transitions is contained within the {@link ConsumerState} objects.
* *
* <h2>State Diagram</h2> * <h2>State Diagram</h2>
* *
* <pre> * <pre>
* +-------------------+ * +-------------------+
* | Waiting on Parent | +------------------+ * | Waiting on Parent | +------------------+
@ -94,14 +94,14 @@ class ConsumerStates {
/** /**
* Represents a the current state of the consumer. This handles the creation of tasks for the consumer, and what to * Represents a the current state of the consumer. This handles the creation of tasks for the consumer, and what to
* do when a transition occurs. * do when a transition occurs.
* *
*/ */
interface ConsumerState { interface ConsumerState {
/** /**
* Creates a new task for this state using the passed in consumer to build the task. If there is no task * Creates a new task for this state using the passed in consumer to build the task. If there is no task
* required for this state it may return a null value. {@link ConsumerState}'s are allowed to modify the * required for this state it may return a null value. {@link ConsumerState}'s are allowed to modify the
* consumer during the execution of this method. * consumer during the execution of this method.
* *
* @param consumer * @param consumer
* the consumer to use build the task, or execute state. * the consumer to use build the task, or execute state.
* @return a valid task for this state or null if there is no task required. * @return a valid task for this state or null if there is no task required.
@ -111,7 +111,7 @@ class ConsumerStates {
/** /**
* Provides the next state of the consumer upon success of the task return by * Provides the next state of the consumer upon success of the task return by
* {@link ConsumerState#createTask(ShardConsumer)}. * {@link ConsumerState#createTask(ShardConsumer)}.
* *
* @return the next state that the consumer should transition to, this may be the same object as the current * @return the next state that the consumer should transition to, this may be the same object as the current
* state. * state.
*/ */
@ -120,7 +120,7 @@ class ConsumerStates {
/** /**
* Provides the next state of the consumer when a shutdown has been requested. The returned state is dependent * Provides the next state of the consumer when a shutdown has been requested. The returned state is dependent
* on the current state, and the shutdown reason. * on the current state, and the shutdown reason.
* *
* @param shutdownReason * @param shutdownReason
* the reason that a shutdown was requested * the reason that a shutdown was requested
* @return the next state that the consumer should transition to, this may be the same object as the current * @return the next state that the consumer should transition to, this may be the same object as the current
@ -131,7 +131,7 @@ class ConsumerStates {
/** /**
* The type of task that {@link ConsumerState#createTask(ShardConsumer)} would return. This is always a valid state * The type of task that {@link ConsumerState#createTask(ShardConsumer)} would return. This is always a valid state
* even if createTask would return a null value. * even if createTask would return a null value.
* *
* @return the type of task that this state represents. * @return the type of task that this state represents.
*/ */
TaskType getTaskType(); TaskType getTaskType();
@ -139,7 +139,7 @@ class ConsumerStates {
/** /**
* An enumeration represent the type of this state. Different consumer states may return the same * An enumeration represent the type of this state. Different consumer states may return the same
* {@link ShardConsumerState}. * {@link ShardConsumerState}.
* *
* @return the type of consumer state this represents. * @return the type of consumer state this represents.
*/ */
ShardConsumerState getState(); ShardConsumerState getState();
@ -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;
} }
@ -136,19 +158,19 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
/** /**
* Sets the checkpoint for a shard and updates ownerSwitchesSinceCheckpoint. * Sets the checkpoint for a shard and updates ownerSwitchesSinceCheckpoint.
* *
* @param shardId shardId to update the checkpoint for * @param shardId shardId to update the checkpoint for
* @param checkpoint checkpoint value to set * @param checkpoint checkpoint value to set
* @param concurrencyToken obtained by calling Lease.getConcurrencyToken for a currently held lease * @param concurrencyToken obtained by calling Lease.getConcurrencyToken for a currently held lease
* *
* @return true if checkpoint update succeeded, false otherwise * @return true if checkpoint update succeeded, false otherwise
* *
* @throws InvalidStateException if lease table does not exist * @throws InvalidStateException if lease table does not exist
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity * @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
* @throws DependencyException if DynamoDB update fails in an unexpected way * @throws DependencyException if DynamoDB update fails in an unexpected way
*/ */
boolean setCheckpoint(String shardId, ExtendedSequenceNumber checkpoint, UUID concurrencyToken) boolean setCheckpoint(String shardId, ExtendedSequenceNumber checkpoint, UUID concurrencyToken)
throws DependencyException, InvalidStateException, ProvisionedThroughputException { throws DependencyException, InvalidStateException, ProvisionedThroughputException {
KinesisClientLease lease = getCurrentlyHeldLease(shardId); KinesisClientLease lease = getCurrentlyHeldLease(shardId);
if (lease == null) { if (lease == null) {
LOG.info(String.format( LOG.info(String.format(
@ -170,7 +192,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
*/ */
@Override @Override
public void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken) public void setCheckpoint(String shardId, ExtendedSequenceNumber checkpointValue, String concurrencyToken)
throws KinesisClientLibException { throws KinesisClientLibException {
try { try {
boolean wasSuccessful = setCheckpoint(shardId, checkpointValue, UUID.fromString(concurrencyToken)); boolean wasSuccessful = setCheckpoint(shardId, checkpointValue, UUID.fromString(concurrencyToken));
if (!wasSuccessful) { if (!wasSuccessful) {
@ -235,8 +257,8 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
*/ */
@Override @Override
public void prepareCheckpoint(String shardId, public void prepareCheckpoint(String shardId,
ExtendedSequenceNumber pendingCheckpointValue, ExtendedSequenceNumber pendingCheckpointValue,
String concurrencyToken) throws KinesisClientLibException { String concurrencyToken) throws KinesisClientLibException {
try { try {
boolean wasSuccessful = boolean wasSuccessful =
prepareCheckpoint(shardId, pendingCheckpointValue, UUID.fromString(concurrencyToken)); prepareCheckpoint(shardId, pendingCheckpointValue, UUID.fromString(concurrencyToken));
@ -307,8 +329,8 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
leaseManager.createLeaseTableIfNotExists(initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity); leaseManager.createLeaseTableIfNotExists(initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity);
if (newTableCreated) { if (newTableCreated) {
LOG.info(String.format( LOG.info(String.format(
"Created new lease table for coordinator with initial read capacity of %d and write capacity of %d.", "Created new lease table for coordinator with initial read capacity of %d and write capacity of %d.",
initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity)); initialLeaseTableReadCapacity, initialLeaseTableWriteCapacity));
} }
// Need to wait for table in active state. // Need to wait for table in active state.
final long secondsBetweenPolls = 10L; final long secondsBetweenPolls = 10L;
@ -321,7 +343,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
/** /**
* Package access for testing. * Package access for testing.
* *
* @throws DependencyException * @throws DependencyException
* @throws InvalidStateException * @throws InvalidStateException
*/ */
@ -331,7 +353,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
/** /**
* Package access for testing. * Package access for testing.
* *
* @throws DependencyException * @throws DependencyException
* @throws InvalidStateException * @throws InvalidStateException
*/ */
@ -342,7 +364,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
/** /**
* Used to get information about leases for Kinesis shards (e.g. sync shards and leases, check on parent shard * Used to get information about leases for Kinesis shards (e.g. sync shards and leases, check on parent shard
* completion). * completion).
* *
* @return LeaseManager * @return LeaseManager
*/ */
ILeaseManager<KinesisClientLease> getLeaseManager() { ILeaseManager<KinesisClientLease> getLeaseManager() {

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,17 +58,20 @@ 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;
@Getter @Getter
private final GetRecordsCache getRecordsCache; private final GetRecordsCache getRecordsCache;
private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher, private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher,
Optional<Integer> retryGetRecordsInSeconds, Optional<Integer> retryGetRecordsInSeconds,
Optional<Integer> maxGetRecordsThreadPool, Optional<Integer> maxGetRecordsThreadPool,
ShardInfo shardInfo) { ShardInfo shardInfo) {
Optional<GetRecordsRetrievalStrategy> getRecordsRetrievalStrategy = retryGetRecordsInSeconds.flatMap(retry -> Optional<GetRecordsRetrievalStrategy> getRecordsRetrievalStrategy = retryGetRecordsInSeconds.flatMap(retry ->
maxGetRecordsThreadPool.map(max -> maxGetRecordsThreadPool.map(max ->
new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, retry, max, shardInfo.getShardId()))); new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, retry, max, shardInfo.getShardId())));
@ -99,20 +102,22 @@ 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,
StreamConfig streamConfig, StreamConfig streamConfig,
ICheckpoint checkpoint, ICheckpoint checkpoint,
IRecordProcessor recordProcessor, IRecordProcessor recordProcessor,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
long parentShardPollIntervalMillis, long parentShardPollIntervalMillis,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
ExecutorService executorService, ExecutorService executorService,
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,23 +148,25 @@ 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,
StreamConfig streamConfig, StreamConfig streamConfig,
ICheckpoint checkpoint, ICheckpoint checkpoint,
IRecordProcessor recordProcessor, IRecordProcessor recordProcessor,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
long parentShardPollIntervalMillis, long parentShardPollIntervalMillis,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
ExecutorService executorService, ExecutorService executorService,
IMetricsFactory metricsFactory, IMetricsFactory metricsFactory,
long backoffTimeMillis, long backoffTimeMillis,
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,
streamConfig, streamConfig,
@ -182,7 +190,8 @@ class ShardConsumer {
new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo), new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo),
retryGetRecordsInSeconds, retryGetRecordsInSeconds,
maxGetRecordsThreadPool, maxGetRecordsThreadPool,
config config,
shardSyncer
); );
} }
@ -203,23 +212,25 @@ 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,
ICheckpoint checkpoint, ICheckpoint checkpoint,
IRecordProcessor recordProcessor, IRecordProcessor recordProcessor,
RecordProcessorCheckpointer recordProcessorCheckpointer, RecordProcessorCheckpointer recordProcessorCheckpointer,
ILeaseManager<KinesisClientLease> leaseManager, ILeaseManager<KinesisClientLease> leaseManager,
long parentShardPollIntervalMillis, long parentShardPollIntervalMillis,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
ExecutorService executorService, ExecutorService executorService,
IMetricsFactory metricsFactory, IMetricsFactory metricsFactory,
long backoffTimeMillis, long backoffTimeMillis,
boolean skipShardSyncAtWorkerInitializationIfLeasesExist, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
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,12 +248,13 @@ 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;
} }
/** /**
* No-op if current task is pending, otherwise submits next task for this shard. * No-op if current task is pending, otherwise submits next task for this shard.
* This method should NOT be called if the ShardConsumer is already in SHUTDOWN_COMPLETED state. * This method should NOT be called if the ShardConsumer is already in SHUTDOWN_COMPLETED state.
* *
* @return true if a new process task was submitted, false otherwise * @return true if a new process task was submitted, false otherwise
*/ */
synchronized boolean consumeShard() { synchronized boolean consumeShard() {
@ -343,7 +355,7 @@ class ShardConsumer {
/** /**
* Requests the shutdown of the this ShardConsumer. This should give the record processor a chance to checkpoint * Requests the shutdown of the this ShardConsumer. This should give the record processor a chance to checkpoint
* before being shutdown. * before being shutdown.
* *
* @param shutdownNotification used to signal that the record processor has been given the chance to shutdown. * @param shutdownNotification used to signal that the record processor has been given the chance to shutdown.
*/ */
void notifyShutdownRequested(ShutdownNotification shutdownNotification) { void notifyShutdownRequested(ShutdownNotification shutdownNotification) {
@ -354,7 +366,7 @@ class ShardConsumer {
/** /**
* Shutdown this ShardConsumer (including invoking the RecordProcessor shutdown API). * Shutdown this ShardConsumer (including invoking the RecordProcessor shutdown API).
* This is called by Worker when it loses responsibility for a shard. * This is called by Worker when it loses responsibility for a shard.
* *
* @return true if shutdown is complete (false if shutdown is still in progress) * @return true if shutdown is complete (false if shutdown is still in progress)
*/ */
synchronized boolean beginShutdown() { synchronized boolean beginShutdown() {
@ -374,7 +386,7 @@ class ShardConsumer {
/** /**
* Used (by Worker) to check if this ShardConsumer instance has been shutdown * Used (by Worker) to check if this ShardConsumer instance has been shutdown
* RecordProcessor shutdown() has been invoked, as appropriate. * RecordProcessor shutdown() has been invoked, as appropriate.
* *
* @return true if shutdown is complete * @return true if shutdown is complete
*/ */
boolean isShutdown() { boolean isShutdown() {
@ -390,7 +402,7 @@ class ShardConsumer {
/** /**
* Figure out next task to run based on current state, task, and shutdown context. * Figure out next task to run based on current state, task, and shutdown context.
* *
* @return Return next task to run * @return Return next task to run
*/ */
private ITask getNextTask() { private ITask getNextTask() {
@ -406,7 +418,7 @@ class ShardConsumer {
/** /**
* Note: This is a private/internal method with package level access solely for testing purposes. * Note: This is a private/internal method with package level access solely for testing purposes.
* Update state based on information about: task success, current state, and shutdown info. * Update state based on information about: task success, current state, and shutdown info.
* *
* @param taskOutcome The outcome of the last task * @param taskOutcome The outcome of the last task
*/ */
void updateState(TaskOutcome taskOutcome) { void updateState(TaskOutcome taskOutcome) {
@ -438,7 +450,7 @@ class ShardConsumer {
/** /**
* Private/Internal method - has package level access solely for testing purposes. * Private/Internal method - has package level access solely for testing purposes.
* *
* @return the currentState * @return the currentState
*/ */
ConsumerStates.ShardConsumerState getCurrentState() { ConsumerStates.ShardConsumerState getCurrentState() {

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,11 +46,12 @@ 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;
/** /**
* Constructor. * Constructor.
* *
* @param kinesisProxy Proxy used to fetch streamInfo (shards) * @param kinesisProxy Proxy used to fetch streamInfo (shards)
* @param leaseManager Lease manager (used to list and create leases for shards) * @param leaseManager Lease manager (used to list and create leases for shards)
* @param initialPositionInStream Initial position in stream * @param initialPositionInStream Initial position in stream
@ -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,26 +51,25 @@ 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,
boolean ignoreUnexpectedChildShards) boolean ignoreUnexpectedChildShards)
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards,
ignoreUnexpectedChildShards); ignoreUnexpectedChildShards);
} }
/** /**
* Check and create leases for any new shards (e.g. following a reshard operation). * Check and create leases for any new shards (e.g. following a reshard operation).
* *
* @param kinesisProxy * @param kinesisProxy
* @param leaseManager * @param leaseManager
* @param initialPositionInStream * @param initialPositionInStream
@ -81,26 +80,18 @@ 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,
boolean ignoreUnexpectedChildShards) boolean ignoreUnexpectedChildShards)
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
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).
* *
* @param kinesisProxy * @param kinesisProxy
* @param leaseManager * @param leaseManager
* @param initialPosition * @param initialPosition
@ -112,12 +103,12 @@ 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,
boolean ignoreUnexpectedChildShards) boolean ignoreUnexpectedChildShards)
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
List<Shard> shards = getShardList(kinesisProxy); List<Shard> shards = getShardList(kinesisProxy);
LOG.debug("Num shards: " + shards.size()); LOG.debug("Num shards: " + shards.size());
@ -131,7 +122,7 @@ class ShardSyncer {
List<KinesisClientLease> currentLeases = leaseManager.listLeases(); List<KinesisClientLease> currentLeases = leaseManager.listLeases();
List<KinesisClientLease> newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition, List<KinesisClientLease> newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition,
inconsistentShardIds); inconsistentShardIds);
LOG.debug("Num new leases to create: " + newLeasesToCreate.size()); LOG.debug("Num new leases to create: " + newLeasesToCreate.size());
for (KinesisClientLease lease : newLeasesToCreate) { for (KinesisClientLease lease : newLeasesToCreate) {
long startTimeMillis = System.currentTimeMillis(); long startTimeMillis = System.currentTimeMillis();
@ -143,10 +134,10 @@ class ShardSyncer {
MetricsHelper.addSuccessAndLatency("CreateLease", startTimeMillis, success, MetricsLevel.DETAILED); MetricsHelper.addSuccessAndLatency("CreateLease", startTimeMillis, success, MetricsLevel.DETAILED);
} }
} }
List<KinesisClientLease> trackedLeases = new ArrayList<>(); List<KinesisClientLease> trackedLeases = new ArrayList<>();
if (currentLeases != null) { if (currentLeases != null) {
trackedLeases.addAll(currentLeases); trackedLeases.addAll(currentLeases);
} }
trackedLeases.addAll(newLeasesToCreate); trackedLeases.addAll(newLeasesToCreate);
cleanupGarbageLeases(shards, trackedLeases, kinesisProxy, leaseManager); cleanupGarbageLeases(shards, trackedLeases, kinesisProxy, leaseManager);
@ -165,13 +156,13 @@ 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, ' ');
throw new KinesisClientLibIOException(String.format("%d open child shards (%s) are inconsistent. " throw new KinesisClientLibIOException(String.format("%d open child shards (%s) are inconsistent. "
+ "This can happen due to a race condition between describeStream and a reshard operation.", + "This can happen due to a race condition between describeStream and a reshard operation.",
inconsistentShardIds.size(), ids)); inconsistentShardIds.size(), ids));
} }
} }
@ -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);
@ -210,28 +201,24 @@ class ShardSyncer {
} }
/** /**
* Note: this has package level access for testing purposes. * Note: this has package level access for testing purposes.
* Useful for asserting that we don't have an incomplete shard list following a reshard operation. * 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 "
+ " while a reshard operation was in progress."; + " while a reshard operation was in progress.";
for (String shardId : shardIdsOfClosedShards) { for (String shardId : shardIdsOfClosedShards) {
Shard shard = shardIdToShardMap.get(shardId); Shard shard = shardIdToShardMap.get(shardId);
if (shard == null) { if (shard == null) {
LOG.info("Shard " + shardId + " is not present in Kinesis anymore."); LOG.info("Shard " + shardId + " is not present in Kinesis anymore.");
continue; continue;
} }
String endingSequenceNumber = shard.getSequenceNumberRange().getEndingSequenceNumber(); String endingSequenceNumber = shard.getSequenceNumberRange().getEndingSequenceNumber();
if (endingSequenceNumber == null) { if (endingSequenceNumber == null) {
throw new KinesisClientLibIOException("Shard " + shardIdsOfClosedShards throw new KinesisClientLibIOException("Shard " + shardIdsOfClosedShards
@ -248,10 +235,10 @@ 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 {
BigInteger startingHashKeyOfClosedShard = new BigInteger(closedShard.getHashKeyRange().getStartingHashKey()); BigInteger startingHashKeyOfClosedShard = new BigInteger(closedShard.getHashKeyRange().getStartingHashKey());
BigInteger endingHashKeyOfClosedShard = new BigInteger(closedShard.getHashKeyRange().getEndingHashKey()); BigInteger endingHashKeyOfClosedShard = new BigInteger(closedShard.getHashKeyRange().getEndingHashKey());
BigInteger minStartingHashKeyOfChildren = null; BigInteger minStartingHashKeyOfChildren = null;
@ -270,23 +257,23 @@ class ShardSyncer {
maxEndingHashKeyOfChildren = endingHashKey; maxEndingHashKeyOfChildren = endingHashKey;
} }
} }
if ((minStartingHashKeyOfChildren == null) || (maxEndingHashKeyOfChildren == null) if ((minStartingHashKeyOfChildren == null) || (maxEndingHashKeyOfChildren == null)
|| (minStartingHashKeyOfChildren.compareTo(startingHashKeyOfClosedShard) > 0) || (minStartingHashKeyOfChildren.compareTo(startingHashKeyOfClosedShard) > 0)
|| (maxEndingHashKeyOfChildren.compareTo(endingHashKeyOfClosedShard) < 0)) { || (maxEndingHashKeyOfChildren.compareTo(endingHashKeyOfClosedShard) < 0)) {
throw new KinesisClientLibIOException("Incomplete shard list: hash key range of shard " throw new KinesisClientLibIOException("Incomplete shard list: hash key range of shard "
+ closedShard.getShardId() + " is not covered by its child shards."); + closedShard.getShardId() + " is not covered by its child shards.");
} }
} }
/** /**
* Helper method to construct shardId->setOfChildShardIds map. * Helper method to construct shardId->setOfChildShardIds map.
* Note: This has package access for testing purposes only. * Note: This has package access for testing purposes only.
* @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()) {
@ -301,7 +288,7 @@ class ShardSyncer {
} }
childShardIds.add(shardId); childShardIds.add(shardId);
} }
String adjacentParentShardId = shard.getAdjacentParentShardId(); String adjacentParentShardId = shard.getAdjacentParentShardId();
if ((adjacentParentShardId != null) && (shardIdToShardMap.containsKey(adjacentParentShardId))) { if ((adjacentParentShardId != null) && (shardIdToShardMap.containsKey(adjacentParentShardId))) {
Set<String> childShardIds = shardIdToChildShardIdsMap.get(adjacentParentShardId); Set<String> childShardIds = shardIdToChildShardIdsMap.get(adjacentParentShardId);
@ -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(
@ -337,13 +324,13 @@ class ShardSyncer {
* we begin processing data from any of its descendants. * we begin processing data from any of its descendants.
* * A shard does not start processing data until data from all its parents has been processed. * * A shard does not start processing data until data from all its parents has been processed.
* Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create * Note, if the initial position is LATEST and a shard has two parents and only one is a descendant - we'll create
* leases corresponding to both the parents - the parent shard which is not a descendant will have * leases corresponding to both the parents - the parent shard which is not a descendant will have
* its checkpoint set to Latest. * its checkpoint set to Latest.
* *
* We assume that if there is an existing lease for a shard, then either: * We assume that if there is an existing lease for a shard, then either:
* * we have previously created a lease for its parent (if it was needed), or * * we have previously created a lease for its parent (if it was needed), or
* * the parent shard has expired. * * the parent shard has expired.
* *
* For example: * For example:
* Shard structure (each level depicts a stream segment): * Shard structure (each level depicts a stream segment):
* 0 1 2 3 4 5 - shards till epoch 102 * 0 1 2 3 4 5 - shards till epoch 102
@ -353,7 +340,7 @@ class ShardSyncer {
* 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber)
* Current leases: (3, 4, 5) * Current leases: (3, 4, 5)
* New leases to create: (2, 6, 7, 8, 9, 10) * New leases to create: (2, 6, 7, 8, 9, 10)
* *
* The leases returned are sorted by the starting sequence number - following the same order * The leases returned are sorted by the starting sequence number - following the same order
* when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail
* before creating all the leases. * before creating all the leases.
@ -363,7 +350,7 @@ class ShardSyncer {
* high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only
* currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases.
* *
* *
* @param shards List of all shards in Kinesis (we'll create new leases based on this set) * @param shards List of all shards in Kinesis (we'll create new leases based on this set)
* @param currentLeases List of current leases * @param currentLeases List of current leases
* @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that
@ -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>();
@ -464,7 +451,7 @@ class ShardSyncer {
* Check if this shard is a descendant of a shard that is (or will be) processed. * Check if this shard is a descendant of a shard that is (or will be) processed.
* Create leases for the ancestors of this shard as required. * Create leases for the ancestors of this shard as required.
* See javadoc of determineNewLeasesToCreate() for rules and example. * See javadoc of determineNewLeasesToCreate() for rules and example.
* *
* @param shardId The shardId to check. * @param shardId The shardId to check.
* @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that
* location in the shard (when an application starts up for the first time - and there are no checkpoints). * location in the shard (when an application starts up for the first time - and there are no checkpoints).
@ -475,13 +462,13 @@ 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,
Map<String, KinesisClientLease> shardIdToLeaseMapOfNewShards, Map<String, KinesisClientLease> shardIdToLeaseMapOfNewShards,
Map<String, Boolean> memoizationContext) { Map<String, Boolean> memoizationContext) {
Boolean previousValue = memoizationContext.get(shardId); Boolean previousValue = memoizationContext.get(shardId);
if (previousValue != null) { if (previousValue != null) {
return previousValue; return previousValue;
@ -530,7 +517,7 @@ class ShardSyncer {
if (descendantParentShardIds.contains(parentShardId) if (descendantParentShardIds.contains(parentShardId)
&& !initialPosition.getInitialPositionInStream() && !initialPosition.getInitialPositionInStream()
.equals(InitialPositionInStream.AT_TIMESTAMP)) { .equals(InitialPositionInStream.AT_TIMESTAMP)) {
lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON); lease.setCheckpoint(ExtendedSequenceNumber.TRIM_HORIZON);
} else { } else {
lease.setCheckpoint(convertToCheckpoint(initialPosition)); lease.setCheckpoint(convertToCheckpoint(initialPosition));
@ -544,7 +531,7 @@ class ShardSyncer {
// after the specified initial position timestamp. // after the specified initial position timestamp.
if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON) if (initialPosition.getInitialPositionInStream().equals(InitialPositionInStream.TRIM_HORIZON)
|| initialPosition.getInitialPositionInStream() || initialPosition.getInitialPositionInStream()
.equals(InitialPositionInStream.AT_TIMESTAMP)) { .equals(InitialPositionInStream.AT_TIMESTAMP)) {
isDescendant = true; isDescendant = true;
} }
} }
@ -561,12 +548,12 @@ class ShardSyncer {
* Helper method to get parent shardIds of the current shard - includes the parent shardIds if: * Helper method to get parent shardIds of the current shard - includes the parent shardIds if:
* a/ they are not null * a/ they are not null
* b/ if they exist in the current shard map (i.e. haven't expired) * b/ if they exist in the current shard map (i.e. haven't expired)
* *
* @param shard Will return parents of this shard * @param shard Will return parents of this shard
* @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)) {
@ -580,40 +567,40 @@ class ShardSyncer {
} }
/** /**
* Delete leases corresponding to shards that no longer exist in the stream. * Delete leases corresponding to shards that no longer exist in the stream.
* Current scheme: Delete a lease if: * Current scheme: Delete a lease if:
* * the corresponding shard is not present in the list of Kinesis shards, AND * * the corresponding shard is not present in the list of Kinesis shards, AND
* * the parentShardIds listed in the lease are also not present in the list of Kinesis shards. * * the parentShardIds listed in the lease are also not present in the list of Kinesis shards.
* @param shards List of all Kinesis shards (assumed to be a consistent snapshot - when stream is in Active state). * @param shards List of all Kinesis shards (assumed to be a consistent snapshot - when stream is in Active state).
* @param trackedLeases List of * @param trackedLeases List of
* @param kinesisProxy Kinesis proxy (used to get shard list) * @param kinesisProxy Kinesis proxy (used to get shard list)
* @param leaseManager * @param leaseManager
* @throws KinesisClientLibIOException Thrown if we couldn't get a fresh shard list from Kinesis. * @throws KinesisClientLibIOException Thrown if we couldn't get a fresh shard list from Kinesis.
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @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)
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException { throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException {
Set<String> kinesisShards = new HashSet<>(); Set<String> kinesisShards = new HashSet<>();
for (Shard shard : shards) { for (Shard shard : shards) {
kinesisShards.add(shard.getShardId()); kinesisShards.add(shard.getShardId());
} }
// 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);
} }
} }
if (!garbageLeases.isEmpty()) { if (!garbageLeases.isEmpty()) {
LOG.info("Found " + garbageLeases.size() LOG.info("Found " + garbageLeases.size()
+ " candidate leases for cleanup. Refreshing list of" + " candidate leases for cleanup. Refreshing list of"
+ " Kinesis shards to pick up recent/latest shards"); + " Kinesis shards to pick up recent/latest shards");
List<Shard> currentShardList = getShardList(kinesisProxy); List<Shard> currentShardList = getShardList(kinesisProxy);
Set<String> currentKinesisShardIds = new HashSet<>(); Set<String> currentKinesisShardIds = new HashSet<>();
@ -622,59 +609,23 @@ 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);
} }
} }
} }
} }
/**
* 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:
* a/ the shard has been fully processed (checkpoint is set to SHARD_END) * a/ the shard has been fully processed (checkpoint is set to SHARD_END)
* b/ we've begun processing all the child shards: we have leases for all child shards and their checkpoint is not * b/ we've begun processing all the child shards: we have leases for all child shards and their checkpoint is not
* TRIM_HORIZON. * TRIM_HORIZON.
* *
* @param currentLeases List of leases we evaluate for clean up * @param currentLeases List of leases we evaluate for clean up
* @param shardIdToShardMap Map of shardId->Shard (assumed to include all Kinesis shards) * @param shardIdToShardMap Map of shardId->Shard (assumed to include all Kinesis shards)
* @param shardIdToChildShardIdsMap Map of shardId->childShardIds (assumed to include all Kinesis shards) * @param shardIdToChildShardIdsMap Map of shardId->childShardIds (assumed to include all Kinesis shards)
@ -685,12 +636,12 @@ 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,
ILeaseManager<KinesisClientLease> leaseManager) ILeaseManager<KinesisClientLease> leaseManager)
throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException {
Set<String> shardIdsOfClosedShards = new HashSet<>(); Set<String> shardIdsOfClosedShards = new HashSet<>();
List<KinesisClientLease> leasesOfClosedShards = new ArrayList<>(); List<KinesisClientLease> leasesOfClosedShards = new ArrayList<>();
for (KinesisClientLease lease : currentLeases) { for (KinesisClientLease lease : currentLeases) {
@ -716,38 +667,38 @@ class ShardSyncer {
cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager); cleanupLeaseForClosedShard(closedShardId, childShardIds, trackedLeaseMap, leaseManager);
} }
} }
} }
} }
/** /**
* Delete lease for the closed shard. Rules for deletion are: * Delete lease for the closed shard. Rules for deletion are:
* a/ the checkpoint for the closed shard is SHARD_END, * a/ the checkpoint for the closed shard is SHARD_END,
* b/ there are leases for all the childShardIds and their checkpoint is NOT TRIM_HORIZON * b/ there are leases for all the childShardIds and their checkpoint is NOT TRIM_HORIZON
* Note: This method has package level access solely for testing purposes. * Note: This method has package level access solely for testing purposes.
* *
* @param closedShardId Identifies the closed shard * @param closedShardId Identifies the closed shard
* @param childShardIds ShardIds of children of the closed shard * @param childShardIds ShardIds of children of the closed shard
* @param trackedLeases shardId->KinesisClientLease map with all leases we are tracking (should not be null) * @param trackedLeases shardId->KinesisClientLease map with all leases we are tracking (should not be null)
* @param leaseManager * @param leaseManager
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @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)
throws DependencyException, InvalidStateException, ProvisionedThroughputException { throws DependencyException, InvalidStateException, ProvisionedThroughputException {
KinesisClientLease leaseForClosedShard = trackedLeases.get(closedShardId); KinesisClientLease leaseForClosedShard = trackedLeases.get(closedShardId);
List<KinesisClientLease> childShardLeases = new ArrayList<>(); List<KinesisClientLease> childShardLeases = new ArrayList<>();
for (String childShardId : childShardIds) { for (String childShardId : childShardIds) {
KinesisClientLease childLease = trackedLeases.get(childShardId); KinesisClientLease childLease = trackedLeases.get(childShardId);
if (childLease != null) { if (childLease != null) {
childShardLeases.add(childLease); childShardLeases.add(childLease);
} }
} }
if ((leaseForClosedShard != null) if ((leaseForClosedShard != null)
&& (leaseForClosedShard.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END)) && (leaseForClosedShard.getCheckpoint().equals(ExtendedSequenceNumber.SHARD_END))
&& (childShardLeases.size() == childShardIds.size())) { && (childShardLeases.size() == childShardIds.size())) {
@ -758,7 +709,7 @@ class ShardSyncer {
break; break;
} }
} }
if (okayToDelete) { if (okayToDelete) {
LOG.info("Deleting lease for shard " + leaseForClosedShard.getLeaseKey() LOG.info("Deleting lease for shard " + leaseForClosedShard.getLeaseKey()
+ " as it has been completely processed and processing of child shards has begun."); + " as it has been completely processed and processing of child shards has begun.");
@ -770,11 +721,11 @@ class ShardSyncer {
/** /**
* Helper method to create a new KinesisClientLease POJO for a shard. * Helper method to create a new KinesisClientLease POJO for a shard.
* Note: Package level access only for testing purposes * Note: Package level access only for testing purposes
* *
* @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);
@ -792,11 +743,11 @@ class ShardSyncer {
/** /**
* Helper method to construct a shardId->Shard map for the specified list of shards. * Helper method to construct a shardId->Shard map for the specified list of shards.
* *
* @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);
@ -807,11 +758,11 @@ class ShardSyncer {
/** /**
* Helper method to return all the open shards for a stream. * Helper method to return all the open shards for a stream.
* Note: Package level access only for testing purposes. * Note: Package level access only for testing purposes.
* *
* @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,9 +774,9 @@ 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)) {
checkpoint = ExtendedSequenceNumber.TRIM_HORIZON; checkpoint = ExtendedSequenceNumber.TRIM_HORIZON;
} else if (position.getInitialPositionInStream().equals(InitialPositionInStream.LATEST)) { } else if (position.getInitialPositionInStream().equals(InitialPositionInStream.LATEST)) {
@ -833,10 +784,10 @@ class ShardSyncer {
} else if (position.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) { } else if (position.getInitialPositionInStream().equals(InitialPositionInStream.AT_TIMESTAMP)) {
checkpoint = ExtendedSequenceNumber.AT_TIMESTAMP; checkpoint = ExtendedSequenceNumber.AT_TIMESTAMP;
} }
return checkpoint; return checkpoint;
} }
/** Helper class to compare leases based on starting sequence number of the corresponding shards. /** Helper class to compare leases based on starting sequence number of the corresponding shards.
* *
*/ */
@ -846,7 +797,7 @@ class ShardSyncer {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
private final Map<String, Shard> shardIdToShardMap; private final Map<String, Shard> shardIdToShardMap;
/** /**
* @param shardIdToShardMapOfAllKinesisShards * @param shardIdToShardMapOfAllKinesisShards
*/ */
@ -860,7 +811,7 @@ class ShardSyncer {
* We assume that lease1 and lease2 are: * We assume that lease1 and lease2 are:
* a/ not null, * a/ not null,
* b/ shards (if found) have non-null starting sequence numbers * b/ shards (if found) have non-null starting sequence numbers
* *
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @Override
@ -870,23 +821,23 @@ class ShardSyncer {
String shardId2 = lease2.getLeaseKey(); String shardId2 = lease2.getLeaseKey();
Shard shard1 = shardIdToShardMap.get(shardId1); Shard shard1 = shardIdToShardMap.get(shardId1);
Shard shard2 = shardIdToShardMap.get(shardId2); Shard shard2 = shardIdToShardMap.get(shardId2);
// If we found shards for the two leases, use comparison of the starting sequence numbers // If we found shards for the two leases, use comparison of the starting sequence numbers
if ((shard1 != null) && (shard2 != null)) { if ((shard1 != null) && (shard2 != null)) {
BigInteger sequenceNumber1 = BigInteger sequenceNumber1 =
new BigInteger(shard1.getSequenceNumberRange().getStartingSequenceNumber()); new BigInteger(shard1.getSequenceNumberRange().getStartingSequenceNumber());
BigInteger sequenceNumber2 = BigInteger sequenceNumber2 =
new BigInteger(shard2.getSequenceNumberRange().getStartingSequenceNumber()); new BigInteger(shard2.getSequenceNumberRange().getStartingSequenceNumber());
result = sequenceNumber1.compareTo(sequenceNumber2); result = sequenceNumber1.compareTo(sequenceNumber2);
} }
if (result == 0) { if (result == 0) {
result = shardId1.compareTo(shardId2); result = shardId1.compareTo(shardId2);
} }
return result; return result;
} }
} }
} }

View file

@ -48,22 +48,24 @@ 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.
*/ */
// CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES
ShutdownTask(ShardInfo shardInfo, ShutdownTask(ShardInfo shardInfo,
IRecordProcessor recordProcessor, IRecordProcessor recordProcessor,
RecordProcessorCheckpointer recordProcessorCheckpointer, RecordProcessorCheckpointer recordProcessorCheckpointer,
ShutdownReason reason, ShutdownReason reason,
IKinesisProxy kinesisProxy, IKinesisProxy kinesisProxy,
InitialPositionInStreamExtended initialPositionInStream, InitialPositionInStreamExtended initialPositionInStream,
boolean cleanupLeasesOfCompletedShards, boolean cleanupLeasesOfCompletedShards,
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,12 +77,13 @@ 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;
} }
/* /*
* Invokes RecordProcessor shutdown() API. * Invokes RecordProcessor shutdown() API.
* (non-Javadoc) * (non-Javadoc)
* *
* @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.ITask#call() * @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.ITask#call()
*/ */
@Override @Override
@ -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(),
@ -395,8 +399,8 @@ public class Worker implements Runnable {
config.getMaxLeasesToStealAtOneTime(), config.getMaxLeasesToStealAtOneTime(),
config.getMaxLeaseRenewalThreads(), config.getMaxLeaseRenewalThreads(),
metricsFactory) metricsFactory)
.withInitialLeaseTableReadCapacity(config.getInitialLeaseTableReadCapacity()) .withInitialLeaseTableReadCapacity(config.getInitialLeaseTableReadCapacity())
.withInitialLeaseTableWriteCapacity(config.getInitialLeaseTableWriteCapacity()), .withInitialLeaseTableWriteCapacity(config.getInitialLeaseTableWriteCapacity()),
execService, execService,
metricsFactory, metricsFactory,
config.getTaskBackoffTimeMillis(), config.getTaskBackoffTimeMillis(),
@ -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) {
@ -457,7 +462,7 @@ public class Worker implements Runnable {
// 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
Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config,
StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis,
long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint, long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint,
KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService,
IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis,
@ -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,16 +508,19 @@ 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
Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, StreamConfig streamConfig, Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, StreamConfig streamConfig,
InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis,
long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint, long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint,
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(),
@ -1294,8 +1317,8 @@ public class Worker implements Runnable {
config.getMaxLeasesToStealAtOneTime(), config.getMaxLeasesToStealAtOneTime(),
config.getMaxLeaseRenewalThreads(), config.getMaxLeaseRenewalThreads(),
metricsFactory) metricsFactory)
.withInitialLeaseTableReadCapacity(config.getInitialLeaseTableReadCapacity()) .withInitialLeaseTableReadCapacity(config.getInitialLeaseTableReadCapacity())
.withInitialLeaseTableWriteCapacity(config.getInitialLeaseTableWriteCapacity()), .withInitialLeaseTableWriteCapacity(config.getInitialLeaseTableWriteCapacity()),
execService, execService,
metricsFactory, metricsFactory,
config.getTaskBackoffTimeMillis(), config.getTaskBackoffTimeMillis(),
@ -1304,14 +1327,15 @@ 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,
final AWSCredentialsProvider credentialsProvider, final AWSCredentialsProvider credentialsProvider,
final ClientConfiguration clientConfiguration, final ClientConfiguration clientConfiguration,
final String endpointUrl, final String endpointUrl,
final String region) { final String region) {
if (credentialsProvider != null) { if (credentialsProvider != null) {
builder.withCredentials(credentialsProvider); builder.withCredentials(credentialsProvider);
} }

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>(
@ -301,8 +370,8 @@ public class LeaseCoordinator<T extends Lease> {
} else { } else {
leaseCoordinatorThreadPool.shutdownNow(); leaseCoordinatorThreadPool.shutdownNow();
LOG.info(String.format("Worker %s stopped lease-tracking threads %dms after stop", LOG.info(String.format("Worker %s stopped lease-tracking threads %dms after stop",
leaseTaker.getWorkerIdentifier(), leaseTaker.getWorkerIdentifier(),
STOP_WAIT_TIME_MILLIS)); STOP_WAIT_TIME_MILLIS));
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.debug("Encountered InterruptedException when awaiting threadpool termination"); LOG.debug("Encountered InterruptedException when awaiting threadpool termination");
@ -328,7 +397,7 @@ public class LeaseCoordinator<T extends Lease> {
/** /**
* Requests that renewals for the given lease are stopped. * Requests that renewals for the given lease are stopped.
* *
* @param lease the lease to stop renewing. * @param lease the lease to stop renewing.
*/ */
public void dropLease(T lease) { public void dropLease(T lease) {
@ -359,7 +428,7 @@ public class LeaseCoordinator<T extends Lease> {
* @throws DependencyException if DynamoDB update fails in an unexpected way * @throws DependencyException if DynamoDB update fails in an unexpected way
*/ */
public boolean updateLease(T lease, UUID concurrencyToken) public boolean updateLease(T lease, UUID concurrencyToken)
throws DependencyException, InvalidStateException, ProvisionedThroughputException { throws DependencyException, InvalidStateException, ProvisionedThroughputException {
return leaseRenewer.updateLease(lease, concurrencyToken); return leaseRenewer.updateLease(lease, concurrencyToken);
} }

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);
} }
@ -122,16 +134,16 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
* Internal implementation of takeLeases. Takes a callable that can provide the time to enable test cases without * Internal implementation of takeLeases. Takes a callable that can provide the time to enable test cases without
* Thread.sleep. Takes a callable instead of a raw time value because the time needs to be computed as-of * Thread.sleep. Takes a callable instead of a raw time value because the time needs to be computed as-of
* immediately after the scan. * immediately after the scan.
* *
* @param timeProvider Callable that will supply the time * @param timeProvider Callable that will supply the time
* *
* @return map of lease key to taken lease * @return map of lease key to taken lease
* *
* @throws DependencyException * @throws DependencyException
* @throws InvalidStateException * @throws InvalidStateException
*/ */
synchronized Map<String, T> takeLeases(Callable<Long> timeProvider) synchronized Map<String, T> takeLeases(Callable<Long> timeProvider)
throws DependencyException, InvalidStateException { throws DependencyException, InvalidStateException {
// Key is leaseKey // Key is leaseKey
Map<String, T> takenLeases = new HashMap<String, T>(); Map<String, T> takenLeases = new HashMap<String, T>();
@ -159,7 +171,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
if (lastException != null) { if (lastException != null) {
LOG.error("Worker " + workerIdentifier LOG.error("Worker " + workerIdentifier
+ " could not scan leases table, aborting takeLeases. Exception caught by last retry:", + " could not scan leases table, aborting takeLeases. Exception caught by last retry:",
lastException); lastException);
return takenLeases; return takenLeases;
} }
@ -235,23 +247,23 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
builder.append(string); builder.append(string);
needDelimiter = true; needDelimiter = true;
} }
return builder.toString(); return builder.toString();
} }
/** /**
* Scan all leases and update lastRenewalTime. Add new leases and delete old leases. * Scan all leases and update lastRenewalTime. Add new leases and delete old leases.
* *
* @param timeProvider callable that supplies the current time * @param timeProvider callable that supplies the current time
* *
* @return list of expired leases, possibly empty, never null. * @return list of expired leases, possibly empty, never null.
* *
* @throws ProvisionedThroughputException if listLeases fails due to lack of provisioned throughput * @throws ProvisionedThroughputException if listLeases fails due to lack of provisioned throughput
* @throws InvalidStateException if the lease table does not exist * @throws InvalidStateException if the lease table does not exist
* @throws DependencyException if listLeases fails in an unexpected way * @throws DependencyException if listLeases fails in an unexpected way
*/ */
private void updateAllLeases(Callable<Long> timeProvider) private void updateAllLeases(Callable<Long> timeProvider)
throws DependencyException, InvalidStateException, ProvisionedThroughputException { throws DependencyException, InvalidStateException, ProvisionedThroughputException {
List<T> freshList = leaseManager.listLeases(); List<T> freshList = leaseManager.listLeases();
try { try {
lastScanTimeNanos = timeProvider.call(); lastScanTimeNanos = timeProvider.call();
@ -322,7 +334,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
/** /**
* Compute the number of leases I should try to take based on the state of the system. * Compute the number of leases I should try to take based on the state of the system.
* *
* @param allLeases map of shardId to lease containing all leases * @param allLeases map of shardId to lease containing all leases
* @param expiredLeases list of leases we determined to be expired * @param expiredLeases list of leases we determined to be expired
* @return set of leases to take. * @return set of leases to take.
@ -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) {
@ -357,8 +369,8 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
int leaseSpillover = Math.max(0, target - maxLeasesForWorker); int leaseSpillover = Math.max(0, target - maxLeasesForWorker);
if (target > maxLeasesForWorker) { if (target > maxLeasesForWorker) {
LOG.warn(String.format("Worker %s target is %d leases and maxLeasesForWorker is %d." LOG.warn(String.format("Worker %s target is %d leases and maxLeasesForWorker is %d."
+ " Resetting target to %d, lease spillover is %d. " + " Resetting target to %d, lease spillover is %d. "
+ " Note that some shards may not be processed if no other workers are able to pick them up.", + " Note that some shards may not be processed if no other workers are able to pick them up.",
workerIdentifier, workerIdentifier,
target, target,
maxLeasesForWorker, maxLeasesForWorker,
@ -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);
@ -401,7 +410,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
if (!leasesToTake.isEmpty()) { if (!leasesToTake.isEmpty()) {
LOG.info(String.format("Worker %s saw %d total leases, %d available leases, %d " LOG.info(String.format("Worker %s saw %d total leases, %d available leases, %d "
+ "workers. Target is %d leases, I have %d leases, I will take %d leases", + "workers. Target is %d leases, I have %d leases, I will take %d leases",
workerIdentifier, workerIdentifier,
numLeases, numLeases,
originalExpiredLeasesSize, originalExpiredLeasesSize,
@ -423,11 +432,11 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
/** /**
* Choose leases to steal by randomly selecting one or more (up to max) from the most loaded worker. * Choose leases to steal by randomly selecting one or more (up to max) from the most loaded worker.
* Stealing rules: * Stealing rules:
* *
* Steal up to maxLeasesToStealAtOneTime leases from the most loaded worker if * Steal up to maxLeasesToStealAtOneTime leases from the most loaded worker if
* a) he has > target leases and I need >= 1 leases : steal min(leases needed, maxLeasesToStealAtOneTime) * a) he has > target leases and I need >= 1 leases : steal min(leases needed, maxLeasesToStealAtOneTime)
* b) he has == target leases and I need > 1 leases : steal 1 * b) he has == target leases and I need > 1 leases : steal 1
* *
* @param leaseCounts map of workerIdentifier to lease count * @param leaseCounts map of workerIdentifier to lease count
* @param needed # of leases needed to reach the target leases for the worker * @param needed # of leases needed to reach the target leases for the worker
* @param target target # of leases per worker * @param target target # of leases per worker
@ -458,7 +467,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
if (numLeasesToSteal <= 0) { if (numLeasesToSteal <= 0) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Worker %s not stealing from most loaded worker %s. He has %d," LOG.debug(String.format("Worker %s not stealing from most loaded worker %s. He has %d,"
+ " target is %d, and I need %d", + " target is %d, and I need %d",
workerIdentifier, workerIdentifier,
mostLoadedWorker.getKey(), mostLoadedWorker.getKey(),
mostLoadedWorker.getValue(), mostLoadedWorker.getValue(),
@ -469,7 +478,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
} else { } else {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Worker %s will attempt to steal %d leases from most loaded worker %s. " LOG.debug(String.format("Worker %s will attempt to steal %d leases from most loaded worker %s. "
+ " He has %d leases, target is %d, I need %d, maxLeasesToSteatAtOneTime is %d.", + " He has %d leases, target is %d, I need %d, maxLeasesToSteatAtOneTime is %d.",
workerIdentifier, workerIdentifier,
numLeasesToSteal, numLeasesToSteal,
mostLoadedWorker.getKey(), mostLoadedWorker.getKey(),
@ -500,7 +509,7 @@ public class LeaseTaker<T extends Lease> implements ILeaseTaker<T> {
/** /**
* Count leases by host. Always includes myself, but otherwise only includes hosts that are currently holding * Count leases by host. Always includes myself, but otherwise only includes hosts that are currently holding
* leases. * leases.
* *
* @param expiredLeases list of leases that are currently expired * @param expiredLeases list of leases that are currently expired
* @return map of workerIdentifier to lease count * @return map of workerIdentifier to lease count
*/ */

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();
} }
@ -210,7 +212,7 @@ public class KinesisClientLibLeaseCoordinatorIntegrationTest {
} }
public void addLeasesToRenew(ILeaseRenewer<KinesisClientLease> renewer, String... shardIds) public void addLeasesToRenew(ILeaseRenewer<KinesisClientLease> renewer, String... shardIds)
throws DependencyException, InvalidStateException { throws DependencyException, InvalidStateException {
List<KinesisClientLease> leasesToRenew = new ArrayList<KinesisClientLease>(); List<KinesisClientLease> leasesToRenew = new ArrayList<KinesisClientLease>();
for (String shardId : shardIds) { for (String shardId : shardIds) {

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,12 +57,13 @@ 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)
public void testSetCheckpointWithUnownedShardId() public void testSetCheckpointWithUnownedShardId()
throws KinesisClientLibException, DependencyException, InvalidStateException, ProvisionedThroughputException { throws KinesisClientLibException, DependencyException, InvalidStateException, ProvisionedThroughputException {
final boolean succeess = leaseCoordinator.setCheckpoint(SHARD_ID, TEST_CHKPT, TEST_UUID); final boolean succeess = leaseCoordinator.setCheckpoint(SHARD_ID, TEST_CHKPT, TEST_UUID);
Assert.assertFalse("Set Checkpoint should return failure", succeess); Assert.assertFalse("Set Checkpoint should return failure", succeess);
leaseCoordinator.setCheckpoint(SHARD_ID, TEST_CHKPT, TEST_UUID.toString()); leaseCoordinator.setCheckpoint(SHARD_ID, TEST_CHKPT, TEST_UUID.toString());
@ -67,7 +71,7 @@ public class KinesisClientLibLeaseCoordinatorTest {
@Test(expected = DependencyException.class) @Test(expected = DependencyException.class)
public void testWaitLeaseTableTimeout() public void testWaitLeaseTableTimeout()
throws DependencyException, ProvisionedThroughputException, IllegalStateException { throws DependencyException, ProvisionedThroughputException, IllegalStateException {
// Set mock lease manager to return false in waiting // Set mock lease manager to return false in waiting
doReturn(false).when(mockLeaseManager).waitUntilLeaseTableExists(anyLong(), anyLong()); doReturn(false).when(mockLeaseManager).waitUntilLeaseTableExists(anyLong(), anyLong());
leaseCoordinator.initialize(); leaseCoordinator.initialize();

View file

@ -97,16 +97,17 @@ 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.
private final ExecutorService executorService = Executors.newFixedThreadPool(1); private final ExecutorService executorService = Executors.newFixedThreadPool(1);
private RecordsFetcherFactory recordsFetcherFactory; private RecordsFetcherFactory recordsFetcherFactory;
private GetRecordsCache getRecordsCache; private GetRecordsCache getRecordsCache;
private KinesisDataFetcher dataFetcher; private KinesisDataFetcher dataFetcher;
@Mock @Mock
private IRecordProcessor processor; private IRecordProcessor processor;
@Mock @Mock
@ -124,12 +125,12 @@ public class ShardConsumerTest {
public void setup() { public void setup() {
getRecordsCache = null; getRecordsCache = null;
dataFetcher = null; dataFetcher = null;
recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory());
when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory);
when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.empty()); when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.empty());
} }
/** /**
* Test method to verify consumer stays in INITIALIZING state when InitializationTask fails. * Test method to verify consumer stays in INITIALIZING state when InitializationTask fails.
*/ */
@ -161,8 +162,9 @@ 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
Thread.sleep(50L); Thread.sleep(50L);
@ -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;
@ -353,7 +357,7 @@ public class ShardConsumerTest {
when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(),
any(IMetricsFactory.class), anyInt())) any(IMetricsFactory.class), anyInt()))
.thenReturn(getRecordsCache); .thenReturn(getRecordsCache);
ShardConsumer consumer = ShardConsumer consumer =
new ShardConsumer(shardInfo, new ShardConsumer(shardInfo,
streamConfig, streamConfig,
@ -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
@ -392,7 +397,7 @@ public class ShardConsumerTest {
} }
Thread.sleep(50L); Thread.sleep(50L);
} }
verify(getRecordsCache, times(5)).getNextResult(); verify(getRecordsCache, times(5)).getNextResult();
assertThat(processor.getShutdownReason(), nullValue()); assertThat(processor.getShutdownReason(), nullValue());
@ -417,7 +422,7 @@ public class ShardConsumerTest {
verify(shutdownNotification, atLeastOnce()).shutdownComplete(); verify(shutdownNotification, atLeastOnce()).shutdownComplete();
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE)));
assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.ZOMBIE))); assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.ZOMBIE)));
verify(getRecordsCache).shutdown(); verify(getRecordsCache).shutdown();
executorService.shutdown(); executorService.shutdown();
@ -497,7 +502,7 @@ public class ShardConsumerTest {
), ),
metricsFactory metricsFactory
); );
ShardConsumer consumer = ShardConsumer consumer =
new ShardConsumer(shardInfo, new ShardConsumer(shardInfo,
streamConfig, streamConfig,
@ -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
@ -615,7 +621,7 @@ public class ShardConsumerTest {
atTimestamp); atTimestamp);
ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON);
RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer(
shardInfo, shardInfo,
checkpoint, checkpoint,
@ -628,7 +634,7 @@ public class ShardConsumerTest {
); );
dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo);
getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords,
new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); new SynchronousGetRecordsRetrievalStrategy(dataFetcher)));
when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(),
@ -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
@ -660,7 +667,7 @@ public class ShardConsumerTest {
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING)));
consumer.consumeShard(); // initialize consumer.consumeShard(); // initialize
Thread.sleep(50L); Thread.sleep(50L);
verify(getRecordsCache).start(); verify(getRecordsCache).start();
// We expect to process all records in numRecs calls // We expect to process all records in numRecs calls
@ -674,7 +681,7 @@ public class ShardConsumerTest {
} }
Thread.sleep(50L); Thread.sleep(50L);
} }
verify(getRecordsCache, times(4)).getNextResult(); verify(getRecordsCache, times(4)).getNextResult();
assertThat(processor.getShutdownReason(), nullValue()); assertThat(processor.getShutdownReason(), nullValue());
@ -692,7 +699,7 @@ public class ShardConsumerTest {
String iterator = fileBasedProxy.getIterator(streamShardId, timestamp); String iterator = fileBasedProxy.getIterator(streamShardId, timestamp);
List<Record> expectedRecords = toUserRecords(fileBasedProxy.get(iterator, numRecs).getRecords()); List<Record> expectedRecords = toUserRecords(fileBasedProxy.get(iterator, numRecs).getRecords());
verifyConsumedRecords(expectedRecords, processor.getProcessedRecords()); verifyConsumedRecords(expectedRecords, processor.getProcessedRecords());
assertEquals(4, processor.getProcessedRecords().size()); assertEquals(4, processor.getProcessedRecords().size());
file.delete(); file.delete();
@ -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());
@ -749,7 +757,7 @@ public class ShardConsumerTest {
Thread.sleep(50L); Thread.sleep(50L);
assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING))); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING)));
} }
@Test @Test
public void testCreateSynchronousGetRecordsRetrieval() { public void testCreateSynchronousGetRecordsRetrieval() {
ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON);
@ -759,7 +767,7 @@ public class ShardConsumerTest {
10, 10,
callProcessRecordsForEmptyRecordList, callProcessRecordsForEmptyRecordList,
skipCheckpointValidationValue, INITIAL_POSITION_LATEST); skipCheckpointValidationValue, INITIAL_POSITION_LATEST);
ShardConsumer shardConsumer = ShardConsumer shardConsumer =
new ShardConsumer(shardInfo, new ShardConsumer(shardInfo,
streamConfig, streamConfig,
@ -774,8 +782,9 @@ 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,24 +813,25 @@ 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);
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
@Test @Test
public void testLongRunningTasks() throws InterruptedException { public void testLongRunningTasks() throws InterruptedException {
final long sleepTime = 1000L; final long sleepTime = 1000L;
ExecutorService mockExecutorService = mock(ExecutorService.class); ExecutorService mockExecutorService = mock(ExecutorService.class);
Future<TaskResult> mockFuture = mock(Future.class); Future<TaskResult> mockFuture = mock(Future.class);
when(mockExecutorService.submit(any(ITask.class))).thenReturn(mockFuture); when(mockExecutorService.submit(any(ITask.class))).thenReturn(mockFuture);
when(mockFuture.isDone()).thenReturn(false); when(mockFuture.isDone()).thenReturn(false);
when(mockFuture.isCancelled()).thenReturn(false); when(mockFuture.isCancelled()).thenReturn(false);
when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.of(sleepTime)); when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.of(sleepTime));
ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.LATEST); ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.LATEST);
StreamConfig streamConfig = new StreamConfig( StreamConfig streamConfig = new StreamConfig(
streamProxy, streamProxy,
@ -830,7 +840,7 @@ public class ShardConsumerTest {
callProcessRecordsForEmptyRecordList, callProcessRecordsForEmptyRecordList,
skipCheckpointValidationValue, skipCheckpointValidationValue,
INITIAL_POSITION_LATEST); INITIAL_POSITION_LATEST);
ShardConsumer shardConsumer = new ShardConsumer( ShardConsumer shardConsumer = new ShardConsumer(
shardInfo, shardInfo,
streamConfig, streamConfig,
@ -843,14 +853,15 @@ 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();
Thread.sleep(sleepTime); Thread.sleep(sleepTime);
shardConsumer.consumeShard(); shardConsumer.consumeShard();
verify(config).getLogWarningForTaskAfterMillis(); verify(config).getLogWarningForTaskAfterMillis();
verify(mockFuture).isDone(); verify(mockFuture).isDone();
verify(mockFuture).isCancelled(); verify(mockFuture).isCancelled();
@ -880,7 +891,7 @@ public class ShardConsumerTest {
} }
Matcher<InitializationInput> initializationInputMatcher(final ExtendedSequenceNumber checkpoint, Matcher<InitializationInput> initializationInputMatcher(final ExtendedSequenceNumber checkpoint,
final ExtendedSequenceNumber pendingCheckpoint) { final ExtendedSequenceNumber pendingCheckpoint) {
return new TypeSafeMatcher<InitializationInput>() { return new TypeSafeMatcher<InitializationInput>() {
@Override @Override
protected boolean matchesSafely(InitializationInput item) { protected boolean matchesSafely(InitializationInput item) {

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
@ -106,7 +107,7 @@ public class ShardSyncTaskIntegrationTest {
/** /**
* Test method for call(). * Test method for call().
* *
* @throws DependencyException * @throws DependencyException
* @throws InvalidStateException * @throws InvalidStateException
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
@ -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);
@ -181,7 +183,7 @@ public class ShardSyncerTest {
/** /**
* Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream)
* *
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @throws InvalidStateException * @throws InvalidStateException
* @throws DependencyException * @throws DependencyException
@ -190,14 +192,14 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testBootstrapShardLeasesAtTrimHorizon() public final void testBootstrapShardLeasesAtTrimHorizon()
throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException, throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException,
KinesisClientLibIOException { KinesisClientLibIOException {
testBootstrapShardLeasesAtStartingPosition(INITIAL_POSITION_TRIM_HORIZON); testBootstrapShardLeasesAtStartingPosition(INITIAL_POSITION_TRIM_HORIZON);
} }
/** /**
* Test bootstrapShardLeases() starting at LATEST (tip of stream) * Test bootstrapShardLeases() starting at LATEST (tip of stream)
* *
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @throws InvalidStateException * @throws InvalidStateException
* @throws DependencyException * @throws DependencyException
@ -206,8 +208,8 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testBootstrapShardLeasesAtLatest() public final void testBootstrapShardLeasesAtLatest()
throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException, throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException,
KinesisClientLibIOException { KinesisClientLibIOException {
testBootstrapShardLeasesAtStartingPosition(INITIAL_POSITION_LATEST); testBootstrapShardLeasesAtStartingPosition(INITIAL_POSITION_LATEST);
} }
@ -220,15 +222,15 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testCheckAndCreateLeasesForNewShardsAtLatest() public final void testCheckAndCreateLeasesForNewShardsAtLatest()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
List<Shard> shards = constructShardListForGraphA(); List<Shard> shards = constructShardListForGraphA();
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); 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");
@ -252,15 +254,15 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizon() public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizon()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
List<Shard> shards = constructShardListForGraphA(); List<Shard> shards = constructShardListForGraphA();
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_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++) {
@ -314,8 +316,8 @@ public class ShardSyncerTest {
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
public final void testCheckAndCreateLeasesForNewShardsWhenParentIsOpen() public final void testCheckAndCreateLeasesForNewShardsWhenParentIsOpen()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
List<Shard> shards = constructShardListForGraphA(); List<Shard> shards = constructShardListForGraphA();
SequenceNumberRange range = shards.get(0).getSequenceNumberRange(); SequenceNumberRange range = shards.get(0).getSequenceNumberRange();
range.setEndingSequenceNumber(null); range.setEndingSequenceNumber(null);
@ -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();
} }
@ -334,8 +336,8 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren() public final void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
List<Shard> shards = constructShardListForGraphA(); List<Shard> shards = constructShardListForGraphA();
Shard shard = shards.get(5); Shard shard = shards.get(5);
Assert.assertEquals("shardId-5", shard.getShardId()); Assert.assertEquals("shardId-5", shard.getShardId());
@ -349,8 +351,8 @@ 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>();
expectedLeaseShardIds.add("shardId-4"); expectedLeaseShardIds.add("shardId-4");
@ -388,8 +390,8 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithDeleteLeaseExceptions() public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithDeleteLeaseExceptions()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
// Define the max calling count for lease manager methods. // Define the max calling count for lease manager methods.
// From the Shard Graph, the max count of calling could be 10 // From the Shard Graph, the max count of calling could be 10
int maxCallingCount = 10; int maxCallingCount = 10;
@ -410,8 +412,8 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithListLeasesExceptions() public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithListLeasesExceptions()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
// Define the max calling count for lease manager methods. // Define the max calling count for lease manager methods.
// From the Shard Graph, the max count of calling could be 10 // From the Shard Graph, the max count of calling could be 10
int maxCallingCount = 10; int maxCallingCount = 10;
@ -432,8 +434,8 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithCreateLeaseExceptions() public final void testCheckAndCreateLeasesForNewShardsAtTrimHorizonAndClosedShardWithCreateLeaseExceptions()
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
// Define the max calling count for lease manager methods. // Define the max calling count for lease manager methods.
// From the Shard Graph, the max count of calling could be 10 // From the Shard Graph, the max count of calling could be 10
int maxCallingCount = 5; int maxCallingCount = 5;
@ -452,7 +454,7 @@ public class ShardSyncerTest {
private void retryCheckAndCreateLeaseForNewShards(IKinesisProxy kinesisProxy, private void retryCheckAndCreateLeaseForNewShards(IKinesisProxy kinesisProxy,
ExceptionThrowingLeaseManagerMethods exceptionMethod, ExceptionThrowingLeaseManagerMethods exceptionMethod,
int exceptionTime, InitialPositionInStreamExtended position) int exceptionTime, InitialPositionInStreamExtended position)
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException { throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException {
if (exceptionMethod != null) { if (exceptionMethod != null) {
ExceptionThrowingLeaseManager exceptionThrowingLeaseManager = ExceptionThrowingLeaseManager exceptionThrowingLeaseManager =
new ExceptionThrowingLeaseManager(leaseManager); new ExceptionThrowingLeaseManager(leaseManager);
@ -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);
} }
} }
@ -569,8 +573,8 @@ public class ShardSyncerTest {
ExceptionThrowingLeaseManagerMethods exceptionMethod, ExceptionThrowingLeaseManagerMethods exceptionMethod,
int exceptionTime, int exceptionTime,
InitialPositionInStreamExtended position) InitialPositionInStreamExtended position)
throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
ExtendedSequenceNumber extendedSequenceNumber = ExtendedSequenceNumber extendedSequenceNumber =
new ExtendedSequenceNumber(position.getInitialPositionInStream().toString()); new ExtendedSequenceNumber(position.getInitialPositionInStream().toString());
List<Shard> shards = constructShardListForGraphA(); List<Shard> shards = constructShardListForGraphA();
@ -617,7 +621,7 @@ public class ShardSyncerTest {
/** /**
* Test bootstrapShardLeases() - cleanup garbage leases. * Test bootstrapShardLeases() - cleanup garbage leases.
* *
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
* @throws InvalidStateException * @throws InvalidStateException
* @throws DependencyException * @throws DependencyException
@ -626,10 +630,10 @@ public class ShardSyncerTest {
*/ */
@Test @Test
public final void testBootstrapShardLeasesCleanupGarbage() public final void testBootstrapShardLeasesCleanupGarbage()
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)));
@ -641,8 +645,8 @@ public class ShardSyncerTest {
} }
private void testBootstrapShardLeasesAtStartingPosition(InitialPositionInStreamExtended initialPosition) private void testBootstrapShardLeasesAtStartingPosition(InitialPositionInStreamExtended initialPosition)
throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException, throws DependencyException, InvalidStateException, ProvisionedThroughputException, IOException,
KinesisClientLibIOException { KinesisClientLibIOException {
List<Shard> shards = new ArrayList<Shard>(); List<Shard> shards = new ArrayList<Shard>();
SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null);
@ -654,8 +658,8 @@ 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());
Set<String> expectedLeaseShardIds = new HashSet<String>(); Set<String> expectedLeaseShardIds = new HashSet<String>();
@ -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,24 +1365,24 @@ 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());
} }
/** /**
* Test isCandidateForCleanup * Test isCandidateForCleanup
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test @Test
@ -1394,28 +1398,28 @@ 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));
} }
/** /**
* Test isCandidateForCleanup * Test isCandidateForCleanup
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
@ -1431,12 +1435,12 @@ 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));
} }
/** /**
* Test isCandidateForCleanup * Test isCandidateForCleanup
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
@ -1452,19 +1456,19 @@ 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));
} }
/** /**
* Test cleanup of lease for a shard that has been fully processed (and processing of child shards has begun). * Test cleanup of lease for a shard that has been fully processed (and processing of child shards has begun).
* *
* @throws DependencyException * @throws DependencyException
* @throws InvalidStateException * @throws InvalidStateException
* @throws ProvisionedThroughputException * @throws ProvisionedThroughputException
*/ */
@Test @Test
public final void testCleanupLeaseForClosedShard() public final void testCleanupLeaseForClosedShard()
throws DependencyException, InvalidStateException, ProvisionedThroughputException { throws DependencyException, InvalidStateException, ProvisionedThroughputException {
String closedShardId = "shardId-2"; String closedShardId = "shardId-2";
KinesisClientLease leaseForClosedShard = newLease(closedShardId); KinesisClientLease leaseForClosedShard = newLease(closedShardId);
leaseForClosedShard.setCheckpoint(new ExtendedSequenceNumber("1234")); leaseForClosedShard.setCheckpoint(new ExtendedSequenceNumber("1234"));
@ -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,33 +1510,33 @@ 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));
} }
/** /**
* Test we can handle trimmed Kinesis shards (absent from the shard list), and valid closed shards. * Test we can handle trimmed Kinesis shards (absent from the shard list), and valid closed shards.
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test @Test
@ -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,13 +1588,13 @@ 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);
} }
/** /**
* Test we throw an exception if the shard is open * Test we throw an exception if the shard is open
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
@ -1602,17 +1606,17 @@ 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);
} }
/** /**
* Test we throw an exception if there are no children * Test we throw an exception if there are no children
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
@ -1624,17 +1628,17 @@ 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);
} }
/** /**
* Test we throw an exception if children don't cover hash key range (min of children > min of parent) * Test we throw an exception if children don't cover hash key range (min of children > min of parent)
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
@ -1647,7 +1651,7 @@ public class ShardSyncerTest {
/** /**
* Test we throw an exception if children don't cover hash key range (max of children < max of parent) * Test we throw an exception if children don't cover hash key range (max of children < max of parent)
* *
* @throws KinesisClientLibIOException * @throws KinesisClientLibIOException
*/ */
@Test(expected = KinesisClientLibIOException.class) @Test(expected = KinesisClientLibIOException.class)
@ -1661,7 +1665,7 @@ public class ShardSyncerTest {
private void testAssertShardCoveredOrAbsentTestIncompleteSplit(HashKeyRange parentHashKeyRange, private void testAssertShardCoveredOrAbsentTestIncompleteSplit(HashKeyRange parentHashKeyRange,
HashKeyRange child1HashKeyRange, HashKeyRange child1HashKeyRange,
HashKeyRange child2HashKeyRange) HashKeyRange child2HashKeyRange)
throws KinesisClientLibIOException { throws KinesisClientLibIOException {
List<Shard> shards = new ArrayList<>(); List<Shard> shards = new ArrayList<>();
String expectedClosedShardId = "shardId-34098"; String expectedClosedShardId = "shardId-34098";
SequenceNumberRange sequenceNumberRange = ShardObjectHelper.newSequenceNumberRange("103", "205"); SequenceNumberRange sequenceNumberRange = ShardObjectHelper.newSequenceNumberRange("103", "205");
@ -1683,17 +1687,17 @@ 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);
} }
/** /**
* Helper method. * Helper method.
* *
* @param shardId * @param shardId
* @return * @return
*/ */

View file

@ -57,7 +57,8 @@ 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;
@ -198,36 +201,36 @@ public class WorkerTest {
private static final com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory SAMPLE_RECORD_PROCESSOR_FACTORY = private static final com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory SAMPLE_RECORD_PROCESSOR_FACTORY =
new com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory() { new com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory() {
@Override
public com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor createProcessor() {
return new com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor() {
@Override @Override
public void shutdown(IRecordProcessorCheckpointer checkpointer, ShutdownReason reason) { public com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor createProcessor() {
if (reason == ShutdownReason.TERMINATE) { return new com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor() {
try {
checkpointer.checkpoint(); @Override
} catch (KinesisClientLibNonRetryableException e) { public void shutdown(IRecordProcessorCheckpointer checkpointer, ShutdownReason reason) {
throw new RuntimeException(e); if (reason == ShutdownReason.TERMINATE) {
try {
checkpointer.checkpoint();
} catch (KinesisClientLibNonRetryableException e) {
throw new RuntimeException(e);
}
}
} }
}
}
@Override @Override
public void processRecords(List<Record> dataRecords, IRecordProcessorCheckpointer checkpointer) { public void processRecords(List<Record> dataRecords, IRecordProcessorCheckpointer checkpointer) {
try { try {
checkpointer.checkpoint(); checkpointer.checkpoint();
} catch (KinesisClientLibNonRetryableException e) { } catch (KinesisClientLibNonRetryableException e) {
throw new RuntimeException(e); throw new RuntimeException(e);
} }
} }
@Override @Override
public void initialize(String shardId) { public void initialize(String shardId) {
}
};
} }
}; };
}
};
private static final IRecordProcessorFactory SAMPLE_RECORD_PROCESSOR_FACTORY_V2 = private static final IRecordProcessorFactory SAMPLE_RECORD_PROCESSOR_FACTORY_V2 =
new V1ToV2RecordProcessorFactoryAdapter(SAMPLE_RECORD_PROCESSOR_FACTORY); new V1ToV2RecordProcessorFactoryAdapter(SAMPLE_RECORD_PROCESSOR_FACTORY);
@ -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);
} }
@ -1495,9 +1498,9 @@ public class WorkerTest {
public void testBuilderWithDefaultKinesisProxy() { public void testBuilderWithDefaultKinesisProxy() {
IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class);
Worker worker = new Worker.Builder() Worker worker = new Worker.Builder()
.recordProcessorFactory(recordProcessorFactory) .recordProcessorFactory(recordProcessorFactory)
.config(config) .config(config)
.build(); .build();
Assert.assertNotNull(worker.getStreamConfig().getStreamProxy()); Assert.assertNotNull(worker.getStreamConfig().getStreamProxy());
Assert.assertTrue(worker.getStreamConfig().getStreamProxy() instanceof KinesisProxy); Assert.assertTrue(worker.getStreamConfig().getStreamProxy() instanceof KinesisProxy);
} }
@ -1508,10 +1511,10 @@ public class WorkerTest {
// Create an instance of KinesisLocalFileProxy for injection and validation // Create an instance of KinesisLocalFileProxy for injection and validation
IKinesisProxy kinesisProxy = mock(KinesisLocalFileProxy.class); IKinesisProxy kinesisProxy = mock(KinesisLocalFileProxy.class);
Worker worker = new Worker.Builder() Worker worker = new Worker.Builder()
.recordProcessorFactory(recordProcessorFactory) .recordProcessorFactory(recordProcessorFactory)
.config(config) .config(config)
.kinesisProxy(kinesisProxy) .kinesisProxy(kinesisProxy)
.build(); .build();
Assert.assertNotNull(worker.getStreamConfig().getStreamProxy()); Assert.assertNotNull(worker.getStreamConfig().getStreamProxy());
Assert.assertTrue(worker.getStreamConfig().getStreamProxy() instanceof KinesisLocalFileProxy); Assert.assertTrue(worker.getStreamConfig().getStreamProxy() instanceof KinesisLocalFileProxy);
} }
@ -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);
} }
@ -2021,11 +2024,11 @@ public class WorkerTest {
} }
private void runAndTestWorker(List<Shard> shardList, private void runAndTestWorker(List<Shard> shardList,
int threadPoolSize, int threadPoolSize,
List<KinesisClientLease> initialLeases, List<KinesisClientLease> initialLeases,
boolean callProcessRecordsForEmptyRecordList, boolean callProcessRecordsForEmptyRecordList,
int numberOfRecordsPerShard, int numberOfRecordsPerShard,
KinesisClientLibConfiguration clientConfig) throws Exception { KinesisClientLibConfiguration clientConfig) throws Exception {
File file = KinesisLocalFileDataCreator.generateTempDataFile(shardList, numberOfRecordsPerShard, "unitTestWT001"); File file = KinesisLocalFileDataCreator.generateTempDataFile(shardList, numberOfRecordsPerShard, "unitTestWT001");
IKinesisProxy fileBasedProxy = new KinesisLocalFileProxy(file.getAbsolutePath()); IKinesisProxy fileBasedProxy = new KinesisLocalFileProxy(file.getAbsolutePath());
@ -2054,15 +2057,15 @@ public class WorkerTest {
} }
private WorkerThread runWorker(List<Shard> shardList, private WorkerThread runWorker(List<Shard> shardList,
List<KinesisClientLease> initialLeases, List<KinesisClientLease> initialLeases,
boolean callProcessRecordsForEmptyRecordList, boolean callProcessRecordsForEmptyRecordList,
long failoverTimeMillis, long failoverTimeMillis,
int numberOfRecordsPerShard, int numberOfRecordsPerShard,
IKinesisProxy kinesisProxy, IKinesisProxy kinesisProxy,
IRecordProcessorFactory recordProcessorFactory, IRecordProcessorFactory recordProcessorFactory,
ExecutorService executorService, ExecutorService executorService,
IMetricsFactory metricsFactory, IMetricsFactory metricsFactory,
KinesisClientLibConfiguration clientConfig) throws Exception { KinesisClientLibConfiguration clientConfig) throws Exception {
final String stageName = "testStageName"; final String stageName = "testStageName";
final int maxRecords = 2; final int maxRecords = 2;
@ -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,
@ -2253,7 +2258,7 @@ public class WorkerTest {
} }
private Map<String, TestStreamlet> private Map<String, TestStreamlet>
findShardIdsAndStreamLetsOfShardsWithOnlyOneProcessor(TestStreamletFactory recordProcessorFactory) { findShardIdsAndStreamLetsOfShardsWithOnlyOneProcessor(TestStreamletFactory recordProcessorFactory) {
Map<String, TestStreamlet> shardIdsAndStreamLetsOfShardsWithOnlyOneProcessor = Map<String, TestStreamlet> shardIdsAndStreamLetsOfShardsWithOnlyOneProcessor =
new HashMap<String, TestStreamlet>(); new HashMap<String, TestStreamlet>();
Set<String> seenShardIds = new HashSet<String>(); Set<String> seenShardIds = new HashSet<String>();

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;
@ -46,8 +47,8 @@ public class LeaseCoordinatorExerciser {
private static final Log LOG = LogFactory.getLog(LeaseCoordinatorExerciser.class); private static final Log LOG = LogFactory.getLog(LeaseCoordinatorExerciser.class);
public static void main(String[] args) public static void main(String[] args)
throws InterruptedException, DependencyException, InvalidStateException, ProvisionedThroughputException, throws InterruptedException, DependencyException, InvalidStateException, ProvisionedThroughputException,
IOException { IOException {
int numCoordinators = 9; int numCoordinators = 9;
int numLeases = 73; int numLeases = 73;
@ -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
@ -119,7 +121,7 @@ public class LeaseTakerIntegrationTest extends LeaseIntegrationTest {
/** /**
* Verify that one activity is stolen from the highest loaded server when a server needs more than one lease and no * Verify that one activity is stolen from the highest loaded server when a server needs more than one lease and no
* expired leases are available. Setup: 4 leases, server foo holds 0, bar holds 1, baz holds 5. * expired leases are available. Setup: 4 leases, server foo holds 0, bar holds 1, baz holds 5.
* *
* Foo should steal from baz. * Foo should steal from baz.
*/ */
@Test @Test