From cdbbff31e810ba8e0eb31df8abbc92d8f13cea35 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Mon, 14 Aug 2017 09:05:12 -0700 Subject: [PATCH 01/35] Advance to 1.8.2-SNAPSHOT (#203) Advanced the version to 1.8.2 --- META-INF/MANIFEST.MF | 2 +- pom.xml | 2 +- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index 826c1ddf..9665aebd 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.1 +Bundle-Version: 1.8.2 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.7 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/pom.xml b/pom.xml index a2bc6511..981f9d70 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.1 + 1.8.2-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 24112ed8..e9673414 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.1"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.2"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From 1ec0b656c9c77a355640b9c32c1cbca0d592e546 Mon Sep 17 00:00:00 2001 From: Walid Baruni Date: Tue, 15 Aug 2017 12:56:32 -0700 Subject: [PATCH 02/35] Add Support for Two Phase Commit (#188) Adds support for making two phase commits. When a checkpoint is started, but fails to complete the next instance of the record processor will receive the attempted checkpoint position at initialization time. --- .../clientlibrary/interfaces/ICheckpoint.java | 27 ++ .../interfaces/IPreparedCheckpointer.java | 41 ++ .../IRecordProcessorCheckpointer.java | 107 +++++ .../lib/checkpoint/Checkpoint.java | 27 ++ .../DoesNothingPreparedCheckpointer.java | 52 ++ .../lib/worker/InitializeTask.java | 7 +- .../KinesisClientLibLeaseCoordinator.java | 72 +++ .../lib/worker/PreparedCheckpointer.java | 51 ++ .../worker/RecordProcessorCheckpointer.java | 135 +++++- .../types/InitializationInput.java | 23 + .../leases/impl/KinesisClientLease.java | 30 +- .../impl/KinesisClientLeaseSerializer.java | 24 + .../checkpoint/CheckpointImplTestBase.java | 62 +++ .../checkpoint/InMemoryCheckpointImpl.java | 18 + .../lib/worker/PreparedCheckpointerTest.java | 49 ++ .../RecordProcessorCheckpointerTest.java | 446 ++++++++++++++++-- .../lib/worker/ShardConsumerTest.java | 88 +++- .../impl/KinesisClientLeaseBuilder.java | 8 +- .../StreamingRecordProcessorTest.java | 29 ++ 19 files changed, 1247 insertions(+), 49 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IPreparedCheckpointer.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/checkpoint/Checkpoint.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointer.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointerTest.java diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/ICheckpoint.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/ICheckpoint.java index d559bfc0..83c29b44 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/ICheckpoint.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/ICheckpoint.java @@ -15,6 +15,7 @@ package com.amazonaws.services.kinesis.clientlibrary.interfaces; import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibException; +import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.Checkpoint; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; /** @@ -46,4 +47,30 @@ public interface ICheckpoint { */ ExtendedSequenceNumber getCheckpoint(String shardId) throws KinesisClientLibException; + /** + * Get the current checkpoint stored for the specified shard, which holds the sequence numbers for the checkpoint + * and pending checkpoint. Useful for checking that the parent shard has been completely processed before we start + * processing the child shard. + * + * @param shardId Current checkpoint for this shard is fetched + * @return Current checkpoint object for this shard, null if there is no record for this shard. + * @throws KinesisClientLibException Thrown if we are unable to fetch the checkpoint + */ + Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException; + + + /** + * Record intent to checkpoint for a shard. Upon failover, the pendingCheckpointValue will be passed to the new + * RecordProcessor's initialize() method. + * + * @param shardId Checkpoint is specified for this shard. + * @param pendingCheckpoint Value of the pending checkpoint (e.g. Kinesis sequence number and subsequence number) + * @param concurrencyToken Used with conditional writes to prevent stale updates + * (e.g. if there was a fail over to a different record processor, we don't want to + * overwrite it's checkpoint) + * @throws KinesisClientLibException Thrown if we were unable to save the checkpoint + */ + void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) + throws KinesisClientLibException; + } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IPreparedCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IPreparedCheckpointer.java new file mode 100644 index 00000000..04827a63 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IPreparedCheckpointer.java @@ -0,0 +1,41 @@ +package com.amazonaws.services.kinesis.clientlibrary.interfaces; + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; + +/** + * Objects of this class are prepared to checkpoint at a specific sequence number. They use an + * IRecordProcessorCheckpointer to do the actual checkpointing, so their checkpoint is subject to the same 'didn't go + * backwards' validation as a normal checkpoint. + */ +public interface IPreparedCheckpointer { + + /** + * @return sequence number of pending checkpoint + */ + ExtendedSequenceNumber getPendingCheckpoint(); + + /** + * This method will record a pending checkpoint. + * + * @throws ThrottlingException Can't store checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this RecordProcessor instance. + * @throws InvalidStateException Can't store checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the checkpoint. The application can + * backoff and retry. + * @throws IllegalArgumentException The sequence number being checkpointed is invalid because it is out of range, + * i.e. it is smaller than the last check point value (prepared or committed), or larger than the greatest + * sequence number seen by the associated record processor. + */ + void checkpoint() + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; + +} \ No newline at end of file diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IRecordProcessorCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IRecordProcessorCheckpointer.java index f64d3c43..df4acc36 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IRecordProcessorCheckpointer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/interfaces/IRecordProcessorCheckpointer.java @@ -120,4 +120,111 @@ public interface IRecordProcessorCheckpointer { void checkpoint(String sequenceNumber, long subSequenceNumber) throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException; + + /** + * This method will record a pending checkpoint at the last data record that was delivered to the record processor. + * If the application fails over between calling prepareCheckpoint() and checkpoint(), the init() method of the next + * IRecordProcessor for this shard will be informed of the prepared sequence number + * + * Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having + * side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete. + * Use the sequence number passed in to init() to behave idempotently. + * + * @return an IPreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this RecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + */ + IPreparedCheckpointer prepareCheckpoint() + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + + /** + * This method will record a pending checkpoint at the at the provided record. This method is analogous to + * {@link #prepareCheckpoint()} but provides the ability to specify the record at which to prepare the checkpoint. + * + * @param record A record at which to prepare checkpoint in this shard. + * + * Application should use this to assist with idempotency across failover by calling prepareCheckpoint before having + * side effects, then by calling checkpoint on the returned PreparedCheckpointer after side effects are complete. + * Use the sequence number and application state passed in to init() to behave idempotently. + * + * @return an IPreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this RecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ + IPreparedCheckpointer prepareCheckpoint(Record record) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException; + + /** + * This method will record a pending checkpoint at the provided sequenceNumber. This method is analogous to + * {@link #prepareCheckpoint()} but provides the ability to specify the sequence number at which to checkpoint. + * + * @param sequenceNumber A sequence number at which to prepare checkpoint in this shard. + + * @return an IPreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this RecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ + IPreparedCheckpointer prepareCheckpoint(String sequenceNumber) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; + + /** + * This method will record a pending checkpoint at the provided sequenceNumber and subSequenceNumber, the latter for + * aggregated records produced with the Producer Library. This method is analogous to {@link #prepareCheckpoint()} + * but provides the ability to specify the sequence number at which to checkpoint + * + * @param sequenceNumber A sequence number at which to prepare checkpoint in this shard. + * @param subSequenceNumber A subsequence number at which to prepare checkpoint within this shard. + * + * @return an IPreparedCheckpointer object that can be called later to persist the checkpoint. + * + * @throws ThrottlingException Can't store pending checkpoint. Can be caused by checkpointing too frequently. + * Consider increasing the throughput/capacity of the checkpoint store or reducing checkpoint frequency. + * @throws ShutdownException The record processor instance has been shutdown. Another instance may have + * started processing some of these records already. + * The application should abort processing via this RecordProcessor instance. + * @throws InvalidStateException Can't store pending checkpoint. + * Unable to store the checkpoint in the DynamoDB table (e.g. table doesn't exist). + * @throws KinesisClientLibDependencyException Encountered an issue when storing the pending checkpoint. The + * application can backoff and retry. + * @throws IllegalArgumentException The sequence number is invalid for one of the following reasons: + * 1.) It appears to be out of range, i.e. it is smaller than the last check point value, or larger than the + * greatest sequence number seen by the associated record processor. + * 2.) It is not a valid sequence number for a record in this shard. + */ + IPreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException; } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/checkpoint/Checkpoint.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/checkpoint/Checkpoint.java new file mode 100644 index 00000000..d81c632f --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/checkpoint/Checkpoint.java @@ -0,0 +1,27 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import lombok.Data; + +/** + * A class encapsulating the 2 pieces of state stored in a checkpoint. + */ +@Data public class Checkpoint { + + private final ExtendedSequenceNumber checkpoint; + private final ExtendedSequenceNumber pendingCheckpoint; + + /** + * Constructor. + * + * @param checkpoint the checkpoint sequence number - cannot be null or empty. + * @param pendingCheckpoint the pending checkpoint sequence number - can be null. + */ + public Checkpoint(ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint) { + if (checkpoint == null || checkpoint.getSequenceNumber().isEmpty()) { + throw new IllegalArgumentException("Checkpoint cannot be null or empty"); + } + this.checkpoint = checkpoint; + this.pendingCheckpoint = pendingCheckpoint; + } +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java new file mode 100644 index 00000000..ed72f317 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java @@ -0,0 +1,52 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException; +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException; +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IPreparedCheckpointer; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; + +/** + * A special IPreparedCheckpointer that does nothing, which can be used when preparing a checkpoint at the current + * checkpoint sequence number where it is never necessary to do another checkpoint. + * This simplifies programming by preventing application developers from having to reason about whether + * their application has processed records before calling prepareCheckpoint + * + * Here's why it's safe to do nothing: + * The only way to checkpoint at current checkpoint value is to have a record processor that gets + * initialized, processes 0 records, then calls prepareCheckpoint(). The value in the table is the same, so there's + * no reason to overwrite it with another copy of itself. + */ +public class DoesNothingPreparedCheckpointer implements IPreparedCheckpointer { + + private final ExtendedSequenceNumber sequenceNumber; + + /** + * Constructor. + * @param sequenceNumber the sequence number value + */ + public DoesNothingPreparedCheckpointer(ExtendedSequenceNumber sequenceNumber) { + this.sequenceNumber = sequenceNumber; + } + + /** + * {@inheritDoc} + */ + @Override + public ExtendedSequenceNumber getPendingCheckpoint() { + return sequenceNumber; + } + + /** + * {@inheritDoc} + */ + @Override + public void checkpoint() + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException, + IllegalArgumentException { + // This method does nothing + } + +} + diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java index 262b98c7..e3d9f607 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java @@ -19,6 +19,7 @@ import org.apache.commons.logging.LogFactory; import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint; import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor; +import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.Checkpoint; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; @@ -75,7 +76,8 @@ class InitializeTask implements ITask { try { LOG.debug("Initializing ShardId " + shardInfo.getShardId()); - ExtendedSequenceNumber initialCheckpoint = checkpoint.getCheckpoint(shardInfo.getShardId()); + Checkpoint initialCheckpointObject = checkpoint.getCheckpointObject(shardInfo.getShardId()); + ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.getCheckpoint(); dataFetcher.initialize(initialCheckpoint.getSequenceNumber(), streamConfig.getInitialPositionInStream()); recordProcessorCheckpointer.setLargestPermittedCheckpointValue(initialCheckpoint); @@ -84,7 +86,8 @@ class InitializeTask implements ITask { LOG.debug("Calling the record processor initialize()."); final InitializationInput initializationInput = new InitializationInput() .withShardId(shardInfo.getShardId()) - .withExtendedSequenceNumber(initialCheckpoint); + .withExtendedSequenceNumber(initialCheckpoint) + .withPendingCheckpointSequenceNumber(initialCheckpointObject.getPendingCheckpoint()); final long recordProcessorStartTimeMillis = System.currentTimeMillis(); try { recordProcessor.initialize(initializationInput); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibLeaseCoordinator.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibLeaseCoordinator.java index 42fa7d0c..448a2953 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibLeaseCoordinator.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibLeaseCoordinator.java @@ -18,6 +18,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -30,6 +31,7 @@ import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException; import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.KinesisClientLibIOException; import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint; +import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.Checkpoint; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.leases.exceptions.DependencyException; import com.amazonaws.services.kinesis.leases.exceptions.InvalidStateException; @@ -157,6 +159,7 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator snB. + * + * @param extendedSequenceNumber the sequence number for the prepared checkpoint + * @return a prepared checkpointer that is ready to checkpoint at the given sequence number. + * @throws KinesisClientLibDependencyException + * @throws InvalidStateException + * @throws ThrottlingException + * @throws ShutdownException + */ + private IPreparedCheckpointer doPrepareCheckpoint(ExtendedSequenceNumber extendedSequenceNumber) + throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException { + + ExtendedSequenceNumber newPrepareCheckpoint = extendedSequenceNumber; + if (sequenceNumberAtShardEnd != null && sequenceNumberAtShardEnd.equals(extendedSequenceNumber)) { + // If we are about to checkpoint the very last sequence number for this shard, we might as well + // just checkpoint at SHARD_END + newPrepareCheckpoint = ExtendedSequenceNumber.SHARD_END; + } + + // Don't actually prepare a checkpoint if they're trying to checkpoint at the current checkpointed value. + // The only way this can happen is if they call prepareCheckpoint() in a record processor that was initialized + // AND that has not processed any records since initialization. + if (newPrepareCheckpoint.equals(lastCheckpointValue)) { + return new DoesNothingPreparedCheckpointer(newPrepareCheckpoint); + } + + try { + checkpoint.prepareCheckpoint(shardInfo.getShardId(), newPrepareCheckpoint, shardInfo.getConcurrencyToken()); + } catch (ThrottlingException | ShutdownException | InvalidStateException + | KinesisClientLibDependencyException e) { + throw e; + } catch (KinesisClientLibException e) { + LOG.warn("Caught exception setting prepareCheckpoint.", e); + throw new KinesisClientLibDependencyException("Caught exception while prepareCheckpointing", e); + } + + PreparedCheckpointer result = new PreparedCheckpointer(newPrepareCheckpoint, this); + return result; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java index 8f044383..a44aa844 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java @@ -23,6 +23,7 @@ public class InitializationInput { private String shardId; private ExtendedSequenceNumber extendedSequenceNumber; + private ExtendedSequenceNumber pendingCheckpointSequenceNumber; /** * Default constructor. @@ -71,4 +72,26 @@ public class InitializationInput { this.extendedSequenceNumber = extendedSequenceNumber; return this; } + + /** + * Get pending checkpoint {@link ExtendedSequenceNumber}. + * + * @return The {@link ExtendedSequenceNumber} in the shard for which a checkpoint is pending + */ + public ExtendedSequenceNumber getPendingCheckpointSequenceNumber() { + return pendingCheckpointSequenceNumber; + } + + /** + * Set pending checkpoint {@link ExtendedSequenceNumber}. + * + * @param pendingCheckpointSequenceNumber The {@link ExtendedSequenceNumber} in the shard for which a checkpoint + * is pending + * @return A reference to this updated object so that method calls can be chained together. + */ + public InitializationInput withPendingCheckpointSequenceNumber( + ExtendedSequenceNumber pendingCheckpointSequenceNumber) { + this.pendingCheckpointSequenceNumber = pendingCheckpointSequenceNumber; + return this; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java index b3a0ce6c..f3b2e828 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java @@ -27,6 +27,7 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber public class KinesisClientLease extends Lease { private ExtendedSequenceNumber checkpoint; + private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet(); @@ -37,16 +38,18 @@ public class KinesisClientLease extends Lease { public KinesisClientLease(KinesisClientLease other) { super(other); this.checkpoint = other.getCheckpoint(); + this.pendingCheckpoint = other.getPendingCheckpoint(); this.ownerSwitchesSinceCheckpoint = other.getOwnerSwitchesSinceCheckpoint(); this.parentShardIds.addAll(other.getParentShardIds()); } KinesisClientLease(String leaseKey, String leaseOwner, Long leaseCounter, UUID concurrencyToken, - Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, Long ownerSwitchesSinceCheckpoint, - Set parentShardIds) { + Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint, + Long ownerSwitchesSinceCheckpoint, Set parentShardIds) { super(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos); this.checkpoint = checkpoint; + this.pendingCheckpoint = pendingCheckpoint; this.ownerSwitchesSinceCheckpoint = ownerSwitchesSinceCheckpoint; this.parentShardIds.addAll(parentShardIds); } @@ -64,6 +67,7 @@ public class KinesisClientLease extends Lease { setOwnerSwitchesSinceCheckpoint(casted.ownerSwitchesSinceCheckpoint); setCheckpoint(casted.checkpoint); + setPendingCheckpoint(casted.pendingCheckpoint); setParentShardIds(casted.parentShardIds); } @@ -75,6 +79,13 @@ public class KinesisClientLease extends Lease { return checkpoint; } + /** + * @return pending checkpoint, possibly null. + */ + public ExtendedSequenceNumber getPendingCheckpoint() { + return pendingCheckpoint; + } + /** * @return count of distinct lease holders between checkpoints. */ @@ -100,6 +111,15 @@ public class KinesisClientLease extends Lease { this.checkpoint = checkpoint; } + /** + * Sets pending checkpoint. + * + * @param pendingCheckpoint can be null + */ + public void setPendingCheckpoint(ExtendedSequenceNumber pendingCheckpoint) { + this.pendingCheckpoint = pendingCheckpoint; + } + /** * Sets ownerSwitchesSinceCheckpoint. * @@ -134,6 +154,7 @@ public class KinesisClientLease extends Lease { final int prime = 31; int result = super.hashCode(); result = prime * result + ((checkpoint == null) ? 0 : checkpoint.hashCode()); + result = pendingCheckpoint == null ? result : prime * result + pendingCheckpoint.hashCode(); result = prime * result + ((ownerSwitchesSinceCheckpoint == null) ? 0 : ownerSwitchesSinceCheckpoint.hashCode()); result = prime * result + ((parentShardIds == null) ? 0 : parentShardIds.hashCode()); @@ -154,6 +175,11 @@ public class KinesisClientLease extends Lease { return false; } else if (!checkpoint.equals(other.checkpoint)) return false; + if (pendingCheckpoint == null) { + if (other.pendingCheckpoint != null) + return false; + } else if (!pendingCheckpoint.equals(other.pendingCheckpoint)) + return false; if (ownerSwitchesSinceCheckpoint == null) { if (other.ownerSwitchesSinceCheckpoint != null) return false; diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 0fad61ea..2383ff10 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -26,6 +26,7 @@ import com.amazonaws.services.dynamodbv2.model.KeySchemaElement; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseSerializer; import com.amazonaws.services.kinesis.leases.util.DynamoUtils; +import com.google.common.base.Strings; /** * An implementation of ILeaseSerializer for KinesisClientLease objects. @@ -35,6 +36,8 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer checkpoints = new HashMap<>(); private Map flushpoints = new HashMap<>(); + private Map pendingCheckpoints = new HashMap<>(); private final String startingSequenceNumber; /** @@ -95,6 +96,7 @@ public class InMemoryCheckpointImpl implements ICheckpoint { throws KinesisClientLibException { checkpoints.put(shardId, checkpointValue); flushpoints.put(shardId, checkpointValue); + pendingCheckpoints.remove(shardId); if (LOG.isDebugEnabled()) { LOG.debug("shardId: " + shardId + " checkpoint: " + checkpointValue); @@ -112,6 +114,22 @@ public class InMemoryCheckpointImpl implements ICheckpoint { return checkpoint; } + @Override + public void prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, String concurrencyToken) + throws KinesisClientLibException { + pendingCheckpoints.put(shardId, pendingCheckpoint); + } + + @Override + public Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException { + ExtendedSequenceNumber checkpoint = flushpoints.get(shardId); + ExtendedSequenceNumber pendingCheckpoint = pendingCheckpoints.get(shardId); + + Checkpoint checkpointObj = new Checkpoint(checkpoint, pendingCheckpoint); + LOG.debug("getCheckpointObject shardId: " + shardId + ", " + checkpointObj); + return checkpointObj; + } + /** Check that string is neither null nor empty. */ static void verifyNotEmpty(String string, String message) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointerTest.java new file mode 100644 index 00000000..bfcd7723 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointerTest.java @@ -0,0 +1,49 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IPreparedCheckpointer; +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +public class PreparedCheckpointerTest { + + /** + * This test verifies the relationship between the constructor and getPendingCheckpoint. + */ + @Test + public void testGetSequenceNumber() { + ExtendedSequenceNumber sn = new ExtendedSequenceNumber("sn"); + IPreparedCheckpointer checkpointer = new PreparedCheckpointer(sn, null); + Assert.assertEquals(sn, checkpointer.getPendingCheckpoint()); + } + + /** + * This test makes sure the PreparedCheckpointer calls the IRecordProcessorCheckpointer properly. + * + * @throws Exception + */ + @Test + public void testCheckpoint() throws Exception { + ExtendedSequenceNumber sn = new ExtendedSequenceNumber("sn"); + IRecordProcessorCheckpointer mockRecordProcessorCheckpointer = Mockito.mock(IRecordProcessorCheckpointer.class); + IPreparedCheckpointer checkpointer = new PreparedCheckpointer(sn, mockRecordProcessorCheckpointer); + checkpointer.checkpoint(); + Mockito.verify(mockRecordProcessorCheckpointer).checkpoint(sn.getSequenceNumber(), sn.getSubSequenceNumber()); + } + + /** + * This test makes sure the PreparedCheckpointer calls the IRecordProcessorCheckpointer properly. + * + * @throws Exception + */ + @Test + public void testDoesNothingPreparedCheckpoint() throws Exception { + ExtendedSequenceNumber sn = new ExtendedSequenceNumber("sn"); + IPreparedCheckpointer checkpointer = new DoesNothingPreparedCheckpointer(sn); + Assert.assertEquals(sn, checkpointer.getPendingCheckpoint()); + // nothing happens here + checkpointer.checkpoint(); + } +} \ No newline at end of file diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java index d5f6b53f..31a1e184 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java @@ -25,12 +25,8 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; -import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException; -import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException; -import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibException; -import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException; -import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException; import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint; +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IPreparedCheckpointer; import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.InMemoryCheckpointImpl; import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.SentinelCheckpoint; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; @@ -49,6 +45,8 @@ public class RecordProcessorCheckpointerTest { private ExtendedSequenceNumber startingExtendedSequenceNumber = new ExtendedSequenceNumber(startingSequenceNumber); private String testConcurrencyToken = "testToken"; private ICheckpoint checkpoint; + private ShardInfo shardInfo; + private SequenceNumberValidator sequenceNumberValidator; private String shardId = "shardId-123"; /** @@ -60,6 +58,9 @@ public class RecordProcessorCheckpointerTest { // A real checkpoint will return a checkpoint value after it is initialized. checkpoint.setCheckpoint(shardId, startingExtendedSequenceNumber, testConcurrencyToken); Assert.assertEquals(this.startingExtendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + + shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); + sequenceNumberValidator = new SequenceNumberValidator(null, shardId, false); } /** @@ -75,8 +76,6 @@ public class RecordProcessorCheckpointerTest { */ @Test public final void testCheckpoint() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - // First call to checkpoint RecordProcessorCheckpointer processingCheckpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, null); @@ -98,9 +97,6 @@ public class RecordProcessorCheckpointerTest { */ @Test public final void testCheckpointRecord() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - SequenceNumberValidator sequenceNumberValidator = - new SequenceNumberValidator(null, shardId, false); RecordProcessorCheckpointer processingCheckpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); @@ -113,13 +109,10 @@ public class RecordProcessorCheckpointerTest { /** * Test method for - * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#checkpoint(UserRecord record)}. + * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#checkpoint(Record record)}. */ @Test public final void testCheckpointSubRecord() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - SequenceNumberValidator sequenceNumberValidator = - new SequenceNumberValidator(null, shardId, false); RecordProcessorCheckpointer processingCheckpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); @@ -137,9 +130,6 @@ public class RecordProcessorCheckpointerTest { */ @Test public final void testCheckpointSequenceNumber() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - SequenceNumberValidator sequenceNumberValidator = - new SequenceNumberValidator(null, shardId, false); RecordProcessorCheckpointer processingCheckpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); @@ -155,9 +145,6 @@ public class RecordProcessorCheckpointerTest { */ @Test public final void testCheckpointExtendedSequenceNumber() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - SequenceNumberValidator sequenceNumberValidator = - new SequenceNumberValidator(null, shardId, false); RecordProcessorCheckpointer processingCheckpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); @@ -167,14 +154,210 @@ public class RecordProcessorCheckpointerTest { Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId)); } + /** + * Test method for + * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#prepareCheckpoint()}. + */ + @Test + public final void testPrepareCheckpoint() throws Exception { + // First call to checkpoint + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + + ExtendedSequenceNumber sequenceNumber1 = new ExtendedSequenceNumber("5001"); + processingCheckpointer.setLargestPermittedCheckpointValue(sequenceNumber1); + IPreparedCheckpointer preparedCheckpoint = processingCheckpointer.prepareCheckpoint(); + Assert.assertEquals(sequenceNumber1, preparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(sequenceNumber1, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // Advance checkpoint + ExtendedSequenceNumber sequenceNumber2 = new ExtendedSequenceNumber("5019"); + + processingCheckpointer.setLargestPermittedCheckpointValue(sequenceNumber2); + preparedCheckpoint = processingCheckpointer.prepareCheckpoint(); + Assert.assertEquals(sequenceNumber2, preparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(sequenceNumber2, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // Checkpoint using preparedCheckpoint + preparedCheckpoint.checkpoint(); + Assert.assertEquals(sequenceNumber2, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(sequenceNumber2, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + + /** + * Test method for + * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#prepareCheckpoint(Record record)}. + */ + @Test + public final void testPrepareCheckpointRecord() throws Exception { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5025"); + Record record = new Record().withSequenceNumber("5025"); + processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); + IPreparedCheckpointer preparedCheckpoint = processingCheckpointer.prepareCheckpoint(record); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, preparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // Checkpoint using preparedCheckpoint + preparedCheckpoint.checkpoint(); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + + /** + * Test method for + * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#prepareCheckpoint(Record record)}. + */ + @Test + public final void testPrepareCheckpointSubRecord() throws Exception { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5030"); + Record record = new Record().withSequenceNumber("5030"); + UserRecord subRecord = new UserRecord(record); + processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); + IPreparedCheckpointer preparedCheckpoint = processingCheckpointer.prepareCheckpoint(subRecord); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, preparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // Checkpoint using preparedCheckpoint + preparedCheckpoint.checkpoint(); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + + /** + * Test method for + * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#checkpoint(String sequenceNumber)}. + */ + @Test + public final void testPrepareCheckpointSequenceNumber() throws Exception { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5035"); + processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); + IPreparedCheckpointer preparedCheckpoint = processingCheckpointer.prepareCheckpoint("5035"); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, preparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // Checkpoint using preparedCheckpoint + preparedCheckpoint.checkpoint(); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + + /** + * Test method for + * {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.RecordProcessorCheckpointer#checkpoint(String sequenceNumber, long subSequenceNumber)}. + */ + @Test + public final void testPrepareCheckpointExtendedSequenceNumber() throws Exception { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5040"); + processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); + IPreparedCheckpointer preparedCheckpoint = processingCheckpointer.prepareCheckpoint("5040", 0); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, preparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // Checkpoint using preparedCheckpoint + preparedCheckpoint.checkpoint(); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + + /** + * Test that having multiple outstanding prepared checkpointers works if they are redeemed in the right order. + */ + @Test + public final void testMultipleOutstandingCheckpointersHappyCase() throws Exception { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + processingCheckpointer.setLargestPermittedCheckpointValue(new ExtendedSequenceNumber("6040")); + + ExtendedSequenceNumber sn1 = new ExtendedSequenceNumber("6010"); + IPreparedCheckpointer firstPreparedCheckpoint = processingCheckpointer.prepareCheckpoint("6010", 0); + Assert.assertEquals(sn1, firstPreparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(sn1, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + ExtendedSequenceNumber sn2 = new ExtendedSequenceNumber("6020"); + IPreparedCheckpointer secondPreparedCheckpoint = processingCheckpointer.prepareCheckpoint("6020", 0); + Assert.assertEquals(sn2, secondPreparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(sn2, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // checkpoint in order + firstPreparedCheckpoint.checkpoint(); + Assert.assertEquals(sn1, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(sn1, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + secondPreparedCheckpoint.checkpoint(); + Assert.assertEquals(sn2, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(sn2, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + + /** + * Test that having multiple outstanding prepared checkpointers works if they are redeemed in the right order. + */ + @Test + public final void testMultipleOutstandingCheckpointersOutOfOrder() throws Exception { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); + processingCheckpointer.setLargestPermittedCheckpointValue(new ExtendedSequenceNumber("7040")); + + ExtendedSequenceNumber sn1 = new ExtendedSequenceNumber("7010"); + IPreparedCheckpointer firstPreparedCheckpoint = processingCheckpointer.prepareCheckpoint("7010", 0); + Assert.assertEquals(sn1, firstPreparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(sn1, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + ExtendedSequenceNumber sn2 = new ExtendedSequenceNumber("7020"); + IPreparedCheckpointer secondPreparedCheckpoint = processingCheckpointer.prepareCheckpoint("7020", 0); + Assert.assertEquals(sn2, secondPreparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(sn2, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // checkpoint out of order + secondPreparedCheckpoint.checkpoint(); + Assert.assertEquals(sn2, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(sn2, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + try { + firstPreparedCheckpoint.checkpoint(); + Assert.fail("checkpoint() should have failed because the sequence number was too low"); + } catch (IllegalArgumentException e) { + } catch (Exception e) { + Assert.fail("checkpoint() should have thrown an IllegalArgumentException but instead threw " + e); + } + } + /** * Test method for update() * */ @Test public final void testUpdate() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - RecordProcessorCheckpointer checkpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, null); ExtendedSequenceNumber sequenceNumber = new ExtendedSequenceNumber("10"); @@ -193,8 +376,6 @@ public class RecordProcessorCheckpointerTest { */ @Test public final void testClientSpecifiedCheckpoint() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - SequenceNumberValidator validator = mock(SequenceNumberValidator.class); Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); RecordProcessorCheckpointer processingCheckpointer = @@ -275,10 +456,129 @@ public class RecordProcessorCheckpointerTest { processingCheckpointer.getLastCheckpointValue()); } + /* + * This test is a mixed test of checking some basic functionality of two phase checkpointing at a sequence number + * and making sure certain bounds checks and validations are being performed inside the checkpointer to prevent + * clients from checkpointing out of order/too big/non-numeric values that aren't valid strings for them to be + * checkpointing + */ + @Test + public final void testClientSpecifiedTwoPhaseCheckpoint() throws Exception { + SequenceNumberValidator validator = mock(SequenceNumberValidator.class); + Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + + // Several checkpoints we're gonna hit + ExtendedSequenceNumber tooSmall = new ExtendedSequenceNumber("2"); + ExtendedSequenceNumber firstSequenceNumber = checkpoint.getCheckpoint(shardId); // 13 + ExtendedSequenceNumber secondSequenceNumber = new ExtendedSequenceNumber("127"); + ExtendedSequenceNumber thirdSequenceNumber = new ExtendedSequenceNumber("5019"); + ExtendedSequenceNumber lastSequenceNumberOfShard = new ExtendedSequenceNumber("6789"); + ExtendedSequenceNumber tooBigSequenceNumber = new ExtendedSequenceNumber("9000"); + + processingCheckpointer.setInitialCheckpointValue(firstSequenceNumber); + processingCheckpointer.setLargestPermittedCheckpointValue(thirdSequenceNumber); + + // confirm that we cannot move backward + try { + processingCheckpointer.prepareCheckpoint(tooSmall.getSequenceNumber(), tooSmall.getSubSequenceNumber()); + Assert.fail("You shouldn't be able to prepare a checkpoint earlier than the initial checkpoint."); + } catch (IllegalArgumentException e) { + // yay! + } + + try { + processingCheckpointer.checkpoint(tooSmall.getSequenceNumber(), tooSmall.getSubSequenceNumber()); + Assert.fail("You shouldn't be able to checkpoint earlier than the initial checkpoint."); + } catch (IllegalArgumentException e) { + // yay! + } + + // advance to first + processingCheckpointer.checkpoint(firstSequenceNumber.getSequenceNumber(), firstSequenceNumber.getSubSequenceNumber()); + Assert.assertEquals(firstSequenceNumber, checkpoint.getCheckpoint(shardId)); + + // prepare checkpoint at initial checkpoint value + IPreparedCheckpointer doesNothingPreparedCheckpoint = + processingCheckpointer.prepareCheckpoint(firstSequenceNumber.getSequenceNumber(), firstSequenceNumber.getSubSequenceNumber()); + Assert.assertTrue(doesNothingPreparedCheckpoint instanceof DoesNothingPreparedCheckpointer); + Assert.assertEquals(firstSequenceNumber, doesNothingPreparedCheckpoint.getPendingCheckpoint()); + Assert.assertEquals(firstSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(firstSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // nothing happens after checkpointing a doesNothingPreparedCheckpoint + doesNothingPreparedCheckpoint.checkpoint(); + Assert.assertEquals(firstSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(firstSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + // advance to second + processingCheckpointer.prepareCheckpoint(secondSequenceNumber.getSequenceNumber(), secondSequenceNumber.getSubSequenceNumber()); + Assert.assertEquals(secondSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + processingCheckpointer.checkpoint(secondSequenceNumber.getSequenceNumber(), secondSequenceNumber.getSubSequenceNumber()); + Assert.assertEquals(secondSequenceNumber, checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + ExtendedSequenceNumber[] valuesWeShouldNotBeAbleToCheckpointAt = + { tooSmall, // Shouldn't be able to move before the first value we ever checkpointed + firstSequenceNumber, // Shouldn't even be able to move back to a once used sequence number + tooBigSequenceNumber, // Can't exceed the max sequence number in the checkpointer + lastSequenceNumberOfShard, // Just another big value that we will use later + null, // Not a valid sequence number + new ExtendedSequenceNumber("bogus-checkpoint-value"), // Can't checkpoint at non-numeric string + ExtendedSequenceNumber.SHARD_END, // Can't go to the end unless it is set as the max + ExtendedSequenceNumber.TRIM_HORIZON, // Can't go back to an initial sentinel value + ExtendedSequenceNumber.LATEST // Can't go back to an initial sentinel value + }; + for (ExtendedSequenceNumber badCheckpointValue : valuesWeShouldNotBeAbleToCheckpointAt) { + try { + processingCheckpointer.prepareCheckpoint(badCheckpointValue.getSequenceNumber(), badCheckpointValue.getSubSequenceNumber()); + fail("checkpointing at bad or out of order sequence didn't throw exception"); + } catch (IllegalArgumentException e) { + + } catch (NullPointerException e) { + + } + Assert.assertEquals("Checkpoint value should not have changed", + secondSequenceNumber, + checkpoint.getCheckpoint(shardId)); + Assert.assertEquals("Last checkpoint value should not have changed", + secondSequenceNumber, + processingCheckpointer.getLastCheckpointValue()); + Assert.assertEquals("Largest sequence number should not have changed", + thirdSequenceNumber, + processingCheckpointer.getLargestPermittedCheckpointValue()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + + } + + // advance to third number + processingCheckpointer.prepareCheckpoint(thirdSequenceNumber.getSequenceNumber(), thirdSequenceNumber.getSubSequenceNumber()); + Assert.assertEquals(thirdSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + processingCheckpointer.checkpoint(thirdSequenceNumber.getSequenceNumber(), thirdSequenceNumber.getSubSequenceNumber()); + Assert.assertEquals(thirdSequenceNumber, checkpoint.getCheckpoint(shardId)); + + // Testing a feature that prevents checkpointing at SHARD_END twice + processingCheckpointer.setLargestPermittedCheckpointValue(lastSequenceNumberOfShard); + processingCheckpointer.setSequenceNumberAtShardEnd(processingCheckpointer.getLargestPermittedCheckpointValue()); + processingCheckpointer.setLargestPermittedCheckpointValue(ExtendedSequenceNumber.SHARD_END); + processingCheckpointer.prepareCheckpoint(lastSequenceNumberOfShard.getSequenceNumber(), lastSequenceNumberOfShard.getSubSequenceNumber()); + Assert.assertEquals("Preparing a checkpoing at the sequence number at the end of a shard should be the same as " + + "preparing a checkpoint at SHARD_END", + ExtendedSequenceNumber.SHARD_END, + checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); + } + private enum CheckpointAction { NONE, NO_SEQUENCE_NUMBER, WITH_SEQUENCE_NUMBER; } + private enum CheckpointerType { + CHECKPOINTER, PREPARED_CHECKPOINTER, PREPARE_THEN_CHECKPOINTER; + } + /** * Tests a bunch of mixed calls between checkpoint() and checkpoint(sequenceNumber) using a helper function. * @@ -290,16 +590,59 @@ public class RecordProcessorCheckpointerTest { @SuppressWarnings("serial") @Test public final void testMixedCheckpointCalls() throws Exception { - ShardInfo shardInfo = new ShardInfo(shardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); - SequenceNumberValidator validator = mock(SequenceNumberValidator.class); Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); - RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + for (LinkedHashMap testPlan : getMixedCallsTestPlan()) { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.CHECKPOINTER); + } + } - List> testPlans = - new ArrayList>(); + /** + * similar to + * {@link RecordProcessorCheckpointerTest#testMixedCheckpointCalls()} , + * but executes in two phase commit mode, where we prepare a checkpoint and then commit the prepared checkpoint + * + * @throws Exception + */ + @SuppressWarnings("serial") + @Test + public final void testMixedTwoPhaseCheckpointCalls() throws Exception { + SequenceNumberValidator validator = mock(SequenceNumberValidator.class); + Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); + + for (LinkedHashMap testPlan : getMixedCallsTestPlan()) { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.PREPARED_CHECKPOINTER); + } + } + + /** + * similar to + * {@link RecordProcessorCheckpointerTest#testMixedCheckpointCalls()} , + * but executes in two phase commit mode, where we prepare a checkpoint, but we checkpoint using the + * RecordProcessorCheckpointer instead of the returned IPreparedCheckpointer + * + * @throws Exception + */ + @SuppressWarnings("serial") + @Test + public final void testMixedTwoPhaseCheckpointCalls2() throws Exception { + SequenceNumberValidator validator = mock(SequenceNumberValidator.class); + Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); + + for (LinkedHashMap testPlan : getMixedCallsTestPlan()) { + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.PREPARE_THEN_CHECKPOINTER); + } + } + + private List> getMixedCallsTestPlan() { + List> testPlans = new ArrayList>(); /* * Simulate a scenario where the checkpointer is created at "latest". @@ -356,11 +699,7 @@ public class RecordProcessorCheckpointerTest { } }); - for (LinkedHashMap testPlan : testPlans) { - processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); - testMixedCheckpointCalls(processingCheckpointer, testPlan); - } + return testPlans; } /** @@ -376,9 +715,11 @@ public class RecordProcessorCheckpointerTest { * @throws Exception */ private void testMixedCheckpointCalls(RecordProcessorCheckpointer processingCheckpointer, - LinkedHashMap checkpointValueAndAction) throws Exception { + LinkedHashMap checkpointValueAndAction, + CheckpointerType checkpointerType) throws Exception { for (Entry entry : checkpointValueAndAction.entrySet()) { + IPreparedCheckpointer preparedCheckpoint = null; ExtendedSequenceNumber lastCheckpointValue = processingCheckpointer.getLastCheckpointValue(); if (SentinelCheckpoint.SHARD_END.toString().equals(entry.getKey())) { @@ -400,10 +741,34 @@ public class RecordProcessorCheckpointerTest { processingCheckpointer.getLastCheckpointValue()); continue; case NO_SEQUENCE_NUMBER: - processingCheckpointer.checkpoint(); + switch (checkpointerType) { + case CHECKPOINTER: + processingCheckpointer.checkpoint(); + break; + case PREPARED_CHECKPOINTER: + preparedCheckpoint = processingCheckpointer.prepareCheckpoint(); + preparedCheckpoint.checkpoint(); + case PREPARE_THEN_CHECKPOINTER: + preparedCheckpoint = processingCheckpointer.prepareCheckpoint(); + processingCheckpointer.checkpoint( + preparedCheckpoint.getPendingCheckpoint().getSequenceNumber(), + preparedCheckpoint.getPendingCheckpoint().getSubSequenceNumber()); + } break; case WITH_SEQUENCE_NUMBER: - processingCheckpointer.checkpoint(entry.getKey()); + switch (checkpointerType) { + case CHECKPOINTER: + processingCheckpointer.checkpoint(entry.getKey()); + break; + case PREPARED_CHECKPOINTER: + preparedCheckpoint = processingCheckpointer.prepareCheckpoint(entry.getKey()); + preparedCheckpoint.checkpoint(); + case PREPARE_THEN_CHECKPOINTER: + preparedCheckpoint = processingCheckpointer.prepareCheckpoint(entry.getKey()); + processingCheckpointer.checkpoint( + preparedCheckpoint.getPendingCheckpoint().getSequenceNumber(), + preparedCheckpoint.getPendingCheckpoint().getSubSequenceNumber()); + } break; } // We must have checkpointed to get here, so let's make sure our last checkpoint value is up to date @@ -413,6 +778,11 @@ public class RecordProcessorCheckpointerTest { Assert.assertEquals("Expected the largest checkpoint value to remain the same since the last set", new ExtendedSequenceNumber(entry.getKey()), processingCheckpointer.getLargestPermittedCheckpointValue()); + + Assert.assertEquals(new ExtendedSequenceNumber(entry.getKey()), checkpoint.getCheckpoint(shardId)); + Assert.assertEquals(new ExtendedSequenceNumber(entry.getKey()), + checkpoint.getCheckpointObject(shardId).getCheckpoint()); + Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); } } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 893f64ed..8073d0df 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.argThat; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; @@ -37,6 +38,7 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; import java.util.ListIterator; +import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -45,6 +47,9 @@ import java.util.concurrent.TimeUnit; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -52,6 +57,7 @@ import org.mockito.runners.MockitoJUnitRunner; import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint; import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor; +import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.Checkpoint; import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.InMemoryCheckpointImpl; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProxy; @@ -108,6 +114,7 @@ public class ShardConsumerTest { ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); when(checkpoint.getCheckpoint(anyString())).thenThrow(NullPointerException.class); + when(checkpoint.getCheckpointObject(anyString())).thenThrow(NullPointerException.class); when(leaseManager.getLease(anyString())).thenReturn(null); StreamConfig streamConfig = @@ -156,6 +163,7 @@ public class ShardConsumerTest { ExecutorService spyExecutorService = spy(executorService); when(checkpoint.getCheckpoint(anyString())).thenThrow(NullPointerException.class); + when(checkpoint.getCheckpointObject(anyString())).thenThrow(NullPointerException.class); when(leaseManager.getLease(anyString())).thenReturn(null); StreamConfig streamConfig = new StreamConfig(streamProxy, @@ -218,8 +226,11 @@ public class ShardConsumerTest { taskBackoffTimeMillis, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); + final ExtendedSequenceNumber pendingCheckpointSequenceNumber = null; when(leaseManager.getLease(anyString())).thenReturn(null); - when(checkpoint.getCheckpoint(anyString())).thenReturn(new ExtendedSequenceNumber("123")); + when(checkpoint.getCheckpointObject(anyString())).thenReturn( + new Checkpoint(checkpointSequenceNumber, pendingCheckpointSequenceNumber)); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // submit BlockOnParentShardTask @@ -233,7 +244,8 @@ public class ShardConsumerTest { consumer.consumeShard(); // submit InitializeTask Thread.sleep(50L); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); - verify(processor, times(1)).initialize(any(InitializationInput.class)); + verify(processor, times(1)).initialize(argThat( + initializationInputMatcher(checkpointSequenceNumber, pendingCheckpointSequenceNumber))); try { // Checking the status of submitted InitializeTask from above should throw exception. @@ -244,14 +256,17 @@ public class ShardConsumerTest { } Thread.sleep(50L); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); - verify(processor, times(1)).initialize(any(InitializationInput.class)); + verify(processor, times(1)).initialize(argThat( + initializationInputMatcher(checkpointSequenceNumber, pendingCheckpointSequenceNumber))); doNothing().when(processor).initialize(any(InitializationInput.class)); consumer.consumeShard(); // submit InitializeTask again. Thread.sleep(50L); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); - verify(processor, times(2)).initialize(any(InitializationInput.class)); + verify(processor, times(2)).initialize(argThat( + initializationInputMatcher(checkpointSequenceNumber, pendingCheckpointSequenceNumber))); + verify(processor, times(2)).initialize(any(InitializationInput.class)); // no other calls with different args // Checking the status of submitted InitializeTask from above should pass. consumer.consumeShard(); @@ -447,6 +462,54 @@ public class ShardConsumerTest { file.delete(); } + @SuppressWarnings("unchecked") + @Test + public final void testConsumeShardInitializedWithPendingCheckpoint() throws Exception { + ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + StreamConfig streamConfig = + new StreamConfig(streamProxy, + 1, + 10, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + ShardConsumer consumer = + new ShardConsumer(shardInfo, + streamConfig, + checkpoint, + processor, + null, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + + final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); + final ExtendedSequenceNumber pendingCheckpointSequenceNumber = new ExtendedSequenceNumber("999"); + when(leaseManager.getLease(anyString())).thenReturn(null); + when(checkpoint.getCheckpointObject(anyString())).thenReturn( + new Checkpoint(checkpointSequenceNumber, pendingCheckpointSequenceNumber)); + + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); + consumer.consumeShard(); // submit BlockOnParentShardTask + Thread.sleep(50L); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); + verify(processor, times(0)).initialize(any(InitializationInput.class)); + + consumer.consumeShard(); // submit InitializeTask + Thread.sleep(50L); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); + verify(processor, times(1)).initialize(argThat( + initializationInputMatcher(checkpointSequenceNumber, pendingCheckpointSequenceNumber))); + verify(processor, times(1)).initialize(any(InitializationInput.class)); // no other calls with different args + + consumer.consumeShard(); + Thread.sleep(50L); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING))); + } + //@formatter:off (gets the formatting wrong) private void verifyConsumedRecords(List expectedRecords, List actualRecords) { @@ -469,4 +532,21 @@ public class ShardConsumerTest { } return userRecords; } + + Matcher initializationInputMatcher(final ExtendedSequenceNumber checkpoint, + final ExtendedSequenceNumber pendingCheckpoint) { + return new TypeSafeMatcher() { + @Override + protected boolean matchesSafely(InitializationInput item) { + return Objects.equals(checkpoint, item.getExtendedSequenceNumber()) + && Objects.equals(pendingCheckpoint, item.getPendingCheckpointSequenceNumber()); + } + + @Override + public void describeTo(Description description) { + description.appendText(String.format("Checkpoint should be %s and pending checkpoint should be %s", + checkpoint, pendingCheckpoint)); + } + }; + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java index 90a1676d..2e8879fe 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseBuilder.java @@ -27,6 +27,7 @@ public class KinesisClientLeaseBuilder { private UUID concurrencyToken; private Long lastCounterIncrementNanos; private ExtendedSequenceNumber checkpoint; + private ExtendedSequenceNumber pendingCheckpoint; private Long ownerSwitchesSinceCheckpoint = 0L; private Set parentShardIds = new HashSet<>(); @@ -60,6 +61,11 @@ public class KinesisClientLeaseBuilder { return this; } + public KinesisClientLeaseBuilder withPendingCheckpoint(ExtendedSequenceNumber pendingCheckpoint) { + this.pendingCheckpoint = pendingCheckpoint; + return this; + } + public KinesisClientLeaseBuilder withOwnerSwitchesSinceCheckpoint(Long ownerSwitchesSinceCheckpoint) { this.ownerSwitchesSinceCheckpoint = ownerSwitchesSinceCheckpoint; return this; @@ -72,6 +78,6 @@ public class KinesisClientLeaseBuilder { public KinesisClientLease build() { return new KinesisClientLease(leaseKey, leaseOwner, leaseCounter, concurrencyToken, lastCounterIncrementNanos, - checkpoint, ownerSwitchesSinceCheckpoint, parentShardIds); + checkpoint, pendingCheckpoint, ownerSwitchesSinceCheckpoint, parentShardIds); } } \ No newline at end of file diff --git a/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java b/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java index d27b9480..d2659349 100644 --- a/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java @@ -18,6 +18,7 @@ import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateExcep import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException; import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException; import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException; +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IPreparedCheckpointer; import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason; @@ -99,6 +100,34 @@ public class StreamingRecordProcessorTest { IllegalArgumentException { throw new UnsupportedOperationException(); } + + @Override + public IPreparedCheckpointer prepareCheckpoint() + throws KinesisClientLibDependencyException, + InvalidStateException, ThrottlingException, ShutdownException { + throw new UnsupportedOperationException(); + } + + @Override + public IPreparedCheckpointer prepareCheckpoint(Record record) + throws KinesisClientLibDependencyException, + InvalidStateException, ThrottlingException, ShutdownException { + throw new UnsupportedOperationException(); + } + + @Override + public IPreparedCheckpointer prepareCheckpoint(String sequenceNumber) + throws KinesisClientLibDependencyException, + InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + throw new UnsupportedOperationException(); + } + + @Override + public IPreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber) + throws KinesisClientLibDependencyException, + InvalidStateException, ThrottlingException, ShutdownException, IllegalArgumentException { + throw new UnsupportedOperationException(); + } }; private MessageWriter messageWriter; From 656b17ceaabd511da5f12cf223468a7e81f48407 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Fri, 15 Sep 2017 11:04:30 -0700 Subject: [PATCH 03/35] Adding logging for DynamoDB ProvisionedThroughputExceededExcpetion (#212) * Adding logging to ProvisionedThroughputExceededException. * Addressing CR comments and changing log level from error to warn * Updated the comments as per cr comments. --- .../com/amazonaws/services/kinesis/leases/impl/LeaseManager.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index 226756eb..6a70eb90 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -564,6 +564,7 @@ public class LeaseManager implements ILeaseManager { protected DependencyException convertAndRethrowExceptions(String operation, String leaseKey, AmazonClientException e) throws ProvisionedThroughputException, InvalidStateException { if (e instanceof ProvisionedThroughputExceededException) { + LOG.warn("Provisioned Throughput on the lease table has been exceeded. It's recommended that you increase the IOPs on the table. Failure to increase the IOPs may cause the application to not make progress."); throw new ProvisionedThroughputException(e); } else if (e instanceof ResourceNotFoundException) { // @formatter:on From 244da44d297c0dc026db813e1099bfceb5d43b17 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Mon, 18 Sep 2017 10:53:08 -0700 Subject: [PATCH 04/35] Allow Configuring GetRecords Calls to Timeout. (#214) It's now possible to configure GetRecords calls to timeout if they take to long. This can be used to terminate a long running request to ensure that record processors continue to make progress This feature was added with contributions from @pfifer, @sahilpalvia, and @BtXin. --- ...ynchronousGetRecordsRetrievalStrategy.java | 126 ++++++++++++++ .../lib/worker/ConsumerStates.java | 5 +- .../worker/GetRecordsRetrievalStrategy.java | 33 ++++ .../worker/KinesisClientLibConfiguration.java | 27 +++ .../clientlibrary/lib/worker/ProcessTask.java | 52 +++++- .../lib/worker/ShardConsumer.java | 40 +++++ ...ynchronousGetRecordsRetrievalStrategy.java | 31 ++++ .../clientlibrary/lib/worker/Worker.java | 67 +++++++- .../KinesisClientLibConfiguratorTest.java | 45 ++++- ...cordsRetrievalStrategyIntegrationTest.java | 156 ++++++++++++++++++ ...ronousGetRecordsRetrievalStrategyTest.java | 137 +++++++++++++++ .../lib/worker/ConsumerStatesTest.java | 39 ++++- .../lib/worker/KinesisDataFetcherTest.java | 11 +- .../lib/worker/ProcessTaskTest.java | 16 +- 14 files changed, 764 insertions(+), 21 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java new file mode 100644 index 00000000..6290dd4f --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java @@ -0,0 +1,126 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; +import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingScope; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import lombok.NonNull; +import lombok.extern.apachecommons.CommonsLog; + +/** + * + */ +@CommonsLog +public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrievalStrategy { + private static final int TIME_TO_KEEP_ALIVE = 5; + private static final int CORE_THREAD_POOL_COUNT = 1; + + private final KinesisDataFetcher dataFetcher; + private final ExecutorService executorService; + private final int retryGetRecordsInSeconds; + private final String shardId; + final CompletionService completionService; + + public AsynchronousGetRecordsRetrievalStrategy(@NonNull final KinesisDataFetcher dataFetcher, + final int retryGetRecordsInSeconds, final int maxGetRecordsThreadPool, String shardId) { + this(dataFetcher, buildExector(maxGetRecordsThreadPool, shardId), retryGetRecordsInSeconds, shardId); + } + + public AsynchronousGetRecordsRetrievalStrategy(final KinesisDataFetcher dataFetcher, + final ExecutorService executorService, final int retryGetRecordsInSeconds, String shardId) { + this(dataFetcher, executorService, retryGetRecordsInSeconds, new ExecutorCompletionService<>(executorService), + shardId); + } + + AsynchronousGetRecordsRetrievalStrategy(KinesisDataFetcher dataFetcher, ExecutorService executorService, + int retryGetRecordsInSeconds, CompletionService completionService, String shardId) { + this.dataFetcher = dataFetcher; + this.executorService = executorService; + this.retryGetRecordsInSeconds = retryGetRecordsInSeconds; + this.completionService = completionService; + this.shardId = shardId; + } + + @Override + public GetRecordsResult getRecords(final int maxRecords) { + if (executorService.isShutdown()) { + throw new IllegalStateException("Strategy has been shutdown"); + } + GetRecordsResult result = null; + Set> futures = new HashSet<>(); + Callable retrieverCall = createRetrieverCallable(maxRecords); + while (true) { + try { + futures.add(completionService.submit(retrieverCall)); + } catch (RejectedExecutionException e) { + log.warn("Out of resources, unable to start additional requests."); + } + + try { + Future resultFuture = completionService.poll(retryGetRecordsInSeconds, + TimeUnit.SECONDS); + if (resultFuture != null) { + result = resultFuture.get(); + break; + } + } catch (ExecutionException e) { + log.error("ExecutionException thrown while trying to get records", e); + } catch (InterruptedException e) { + log.error("Thread was interrupted", e); + break; + } + } + futures.stream().peek(f -> f.cancel(true)).filter(Future::isCancelled).forEach(f -> { + try { + completionService.take(); + } catch (InterruptedException e) { + log.error("Exception thrown while trying to empty the threadpool."); + } + }); + return result; + } + + private Callable createRetrieverCallable(int maxRecords) { + ThreadSafeMetricsDelegatingScope metricsScope = new ThreadSafeMetricsDelegatingScope(MetricsHelper.getMetricsScope()); + return () -> { + try { + MetricsHelper.setMetricsScope(metricsScope); + return dataFetcher.getRecords(maxRecords); + } finally { + MetricsHelper.unsetMetricsScope(); + } + }; + } + + @Override + public void shutdown() { + executorService.shutdownNow(); + } + + @Override + public boolean isShutdown() { + return executorService.isShutdown(); + } + + private static ExecutorService buildExector(int maxGetRecordsThreadPool, String shardId) { + String threadNameFormat = "get-records-worker-" + shardId + "-%d"; + return new ThreadPoolExecutor(CORE_THREAD_POOL_COUNT, maxGetRecordsThreadPool, TIME_TO_KEEP_ALIVE, + TimeUnit.SECONDS, new LinkedBlockingQueue<>(1), + new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadNameFormat).build(), + new ThreadPoolExecutor.AbortPolicy()); + } +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index d967b2c3..f6d96b4d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -14,6 +14,8 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import java.util.Optional; + /** * Top level container for all the possible states a {@link ShardConsumer} can be in. The logic for creation of tasks, * and state transitions is contained within the {@link ConsumerState} objects. @@ -309,7 +311,8 @@ class ConsumerStates { public ITask createTask(ShardConsumer consumer) { return new ProcessTask(consumer.getShardInfo(), consumer.getStreamConfig(), consumer.getRecordProcessor(), consumer.getRecordProcessorCheckpointer(), consumer.getDataFetcher(), - consumer.getTaskBackoffTimeMillis(), consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist()); + consumer.getTaskBackoffTimeMillis(), consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist(), + consumer.getRetryGetRecordsInSeconds(), consumer.getMaxGetRecordsThreadPool()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java new file mode 100644 index 00000000..a391ac59 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java @@ -0,0 +1,33 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; + +/** + * Represents a strategy to retrieve records from Kinesis. Allows for variations on how records are retrieved from + * Kinesis. + */ +public interface GetRecordsRetrievalStrategy { + /** + * Gets a set of records from Kinesis. + * + * @param maxRecords + * passed to Kinesis, and can be used to restrict the number of records returned from Kinesis. + * @return the resulting records. + * @throws IllegalStateException + * if the strategy has been shutdown. + */ + GetRecordsResult getRecords(int maxRecords); + + /** + * Releases any resources used by the strategy. Once the strategy is shutdown it is no longer safe to call + * {@link #getRecords(int)}. + */ + void shutdown(); + + /** + * Returns whether this strategy has been shutdown. + * + * @return true if the strategy has been shutdown, false otherwise. + */ + boolean isShutdown(); +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index e9673414..62d87f30 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -217,6 +217,12 @@ public class KinesisClientLibConfiguration { @Getter private Optional timeoutInSeconds = Optional.empty(); + @Getter + private Optional retryGetRecordsInSeconds = Optional.empty(); + + @Getter + private Optional maxGetRecordsThreadPool = Optional.empty(); + @Getter private int maxLeaseRenewalThreads = DEFAULT_MAX_LEASE_RENEWAL_THREADS; @@ -1111,6 +1117,27 @@ public class KinesisClientLibConfiguration { return this; } + + /** + * @param retryGetRecordsInSeconds the time in seconds to wait before the worker retries to get a record. + * @return this configuration object. + */ + public KinesisClientLibConfiguration withRetryGetRecordsInSeconds(final int retryGetRecordsInSeconds) { + checkIsValuePositive("retryGetRecordsInSeconds", retryGetRecordsInSeconds); + this.retryGetRecordsInSeconds = Optional.of(retryGetRecordsInSeconds); + return this; + } + + /** + *@param maxGetRecordsThreadPool the max number of threads in the getRecords thread pool. + *@return this configuration object + */ + public KinesisClientLibConfiguration withMaxGetRecordsThreadPool(final int maxGetRecordsThreadPool) { + checkIsValuePositive("maxGetRecordsThreadPool", maxGetRecordsThreadPool); + this.maxGetRecordsThreadPool = Optional.of(maxGetRecordsThreadPool); + return this; + } + /** * @param timeoutInSeconds The timeout in seconds to wait for the MultiLangProtocol to wait for */ diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java index c419c693..223236f6 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java @@ -18,6 +18,7 @@ import java.math.BigInteger; import java.util.Collections; import java.util.List; import java.util.ListIterator; +import java.util.Optional; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -62,6 +63,19 @@ class ProcessTask implements ITask { private final Shard shard; private final ThrottlingReporter throttlingReporter; + private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + + private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher, + Optional retryGetRecordsInSeconds, + Optional maxGetRecordsThreadPool, + ShardInfo shardInfo) { + Optional getRecordsRetrievalStrategy = retryGetRecordsInSeconds.flatMap(retry -> + maxGetRecordsThreadPool.map(max -> + new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, retry, max, shardInfo.getShardId()))); + + return getRecordsRetrievalStrategy.orElse(new SynchronousGetRecordsRetrievalStrategy(dataFetcher)); + } + /** * @param shardInfo * contains information about the shard @@ -77,11 +91,38 @@ class ProcessTask implements ITask { * backoff time when catching exceptions */ public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, - RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, - long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist) { + RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, + long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist) { + this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis, + skipShardSyncAtWorkerInitializationIfLeasesExist, Optional.empty(), Optional.empty()); + } + + /** + * @param shardInfo + * contains information about the shard + * @param streamConfig + * Stream configuration + * @param recordProcessor + * Record processor used to process the data records for the shard + * @param recordProcessorCheckpointer + * Passed to the RecordProcessor so it can checkpoint progress + * @param dataFetcher + * Kinesis data fetcher (used to fetch records from Kinesis) + * @param backoffTimeMillis + * backoff time when catching exceptions + * @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. + */ + public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, + RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, + long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, + Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool) { this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, - new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId())); + new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()), + makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo)); } /** @@ -103,7 +144,7 @@ class ProcessTask implements ITask { public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, - ThrottlingReporter throttlingReporter) { + ThrottlingReporter throttlingReporter, GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { super(); this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; @@ -113,6 +154,7 @@ class ProcessTask implements ITask { this.backoffTimeMillis = backoffTimeMillis; this.throttlingReporter = throttlingReporter; IKinesisProxy kinesisProxy = this.streamConfig.getStreamProxy(); + this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; // If skipShardSyncAtWorkerInitializationIfLeasesExist is set, we will not get the shard for // this ProcessTask. In this case, duplicate KPL user records in the event of resharding will // not be dropped during deaggregation of Amazon Kinesis records. This is only applicable if @@ -368,7 +410,7 @@ class ProcessTask implements ITask { * @return list of data records from Kinesis */ private GetRecordsResult getRecordsResultAndRecordMillisBehindLatest() { - final GetRecordsResult getRecordsResult = dataFetcher.getRecords(streamConfig.getMaxRecords()); + final GetRecordsResult getRecordsResult = getRecordsRetrievalStrategy.getRecords(streamConfig.getMaxRecords()); if (getRecordsResult == null) { // Stream no longer exists diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 63cce40d..70a81fbc 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -15,10 +15,12 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; +import lombok.Getter; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -53,6 +55,10 @@ class ShardConsumer { private final boolean cleanupLeasesOfCompletedShards; private final long taskBackoffTimeMillis; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; + @Getter + private final Optional retryGetRecordsInSeconds; + @Getter + private final Optional maxGetRecordsThreadPool; private ITask currentTask; private long currentTaskSubmitTime; @@ -93,6 +99,38 @@ class ShardConsumer { IMetricsFactory metricsFactory, long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist) { + this(shardInfo, streamConfig, checkpoint,recordProcessor, leaseManager, parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, executorService, metricsFactory, backoffTimeMillis, + skipShardSyncAtWorkerInitializationIfLeasesExist, Optional.empty(), Optional.empty()); + } + + /** + * @param shardInfo Shard information + * @param streamConfig Stream configuration to use + * @param checkpoint Checkpoint tracker + * @param recordProcessor Record processor used to process the data records for the shard + * @param leaseManager Used to create leases for new shards + * @param parentShardPollIntervalMillis Wait for this long if parent shards are not done (or we get an exception) + * @param executorService ExecutorService used to execute process tasks for this shard + * @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard + * @param backoffTimeMillis backoff interval when we encounter exceptions + * @param 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. + */ + // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES + ShardConsumer(ShardInfo shardInfo, + StreamConfig streamConfig, + ICheckpoint checkpoint, + IRecordProcessor recordProcessor, + ILeaseManager leaseManager, + long parentShardPollIntervalMillis, + boolean cleanupLeasesOfCompletedShards, + ExecutorService executorService, + IMetricsFactory metricsFactory, + long backoffTimeMillis, + boolean skipShardSyncAtWorkerInitializationIfLeasesExist, + Optional retryGetRecordsInSeconds, + Optional maxGetRecordsThreadPool) { this.streamConfig = streamConfig; this.recordProcessor = recordProcessor; this.executorService = executorService; @@ -111,6 +149,8 @@ class ShardConsumer { this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards; this.taskBackoffTimeMillis = backoffTimeMillis; this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist; + this.retryGetRecordsInSeconds = retryGetRecordsInSeconds; + this.maxGetRecordsThreadPool = maxGetRecordsThreadPool; } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java new file mode 100644 index 00000000..77a60448 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java @@ -0,0 +1,31 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import lombok.Data; +import lombok.NonNull; + +/** + * + */ +@Data +public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrievalStrategy { + @NonNull + private final KinesisDataFetcher dataFetcher; + + @Override + public GetRecordsResult getRecords(final int maxRecords) { + return dataFetcher.getRecords(maxRecords); + } + + @Override + public void shutdown() { + // + // Does nothing as this retriever doesn't manage any resources + // + } + + @Override + public boolean isShutdown() { + return false; + } +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index fd461e31..3cfb9f2f 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.Collection; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -85,6 +86,9 @@ public class Worker implements Runnable { private final long taskBackoffTimeMillis; private final long failoverTimeMillis; + private final Optional retryGetRecordsInSeconds; + private final Optional maxGetRecordsThreadPool; + // private final KinesisClientLeaseManager leaseManager; private final KinesisClientLibLeaseCoordinator leaseCoordinator; private final ShardSyncTaskManager controlServer; @@ -266,7 +270,9 @@ public class Worker implements Runnable { config.getTaskBackoffTimeMillis(), config.getFailoverTimeMillis(), config.getSkipShardSyncAtWorkerInitializationIfLeasesExist(), - config.getShardPrioritizationStrategy()); + config.getShardPrioritizationStrategy(), + config.getRetryGetRecordsInSeconds(), + config.getMaxGetRecordsThreadPool()); // If a region name was explicitly specified, use it as the region for Amazon Kinesis and Amazon DynamoDB. if (config.getRegionName() != null) { @@ -333,6 +339,56 @@ public class Worker implements Runnable { KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization) { + this(applicationName, recordProcessorFactory, streamConfig, initialPositionInStream, parentShardPollIntervalMillis, + shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService, + metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, + shardPrioritization, Optional.empty(), Optional.empty()); + } + + + /** + * @param applicationName + * Name of the Kinesis application + * @param recordProcessorFactory + * Used to get record processor instances for processing data from shards + * @param streamConfig + * Stream configuration + * @param initialPositionInStream + * One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. The KinesisClientLibrary will start fetching data from + * this location in the stream when an application starts up for the first time and there are no + * checkpoints. If there are checkpoints, we start from the checkpoint position. + * @param parentShardPollIntervalMillis + * Wait for this long between polls to check if parent shards are done + * @param shardSyncIdleTimeMillis + * Time between tasks to sync leases and Kinesis shards + * @param cleanupLeasesUponShardCompletion + * Clean up shards we've finished processing (don't wait till they expire in Kinesis) + * @param checkpoint + * Used to get/set checkpoints + * @param leaseCoordinator + * Lease coordinator (coordinates currently owned leases) + * @param execService + * ExecutorService to use for processing records (support for multi-threaded consumption) + * @param metricsFactory + * Metrics factory used to emit metrics + * @param taskBackoffTimeMillis + * Backoff period when tasks encounter an exception + * @param shardPrioritization + * Provides prioritization logic to decide which available shards process first + * @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. + */ + // NOTE: This has package level access solely for testing + // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES + Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, StreamConfig streamConfig, + InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, + long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint, + KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, + IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, + boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization, + Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool) { this.applicationName = applicationName; this.recordProcessorFactory = recordProcessorFactory; this.streamConfig = streamConfig; @@ -351,8 +407,11 @@ public class Worker implements Runnable { this.failoverTimeMillis = failoverTimeMillis; this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist; this.shardPrioritization = shardPrioritization; + this.retryGetRecordsInSeconds = retryGetRecordsInSeconds; + this.maxGetRecordsThreadPool = maxGetRecordsThreadPool; } + /** * @return the applicationName */ @@ -786,7 +845,7 @@ public class Worker implements Runnable { return new ShardConsumer(shardInfo, streamConfig, checkpointTracker, recordProcessor, leaseCoordinator.getLeaseManager(), parentShardPollIntervalMillis, cleanupLeasesUponShardCompletion, executorService, metricsFactory, taskBackoffTimeMillis, - skipShardSyncAtWorkerInitializationIfLeasesExist); + skipShardSyncAtWorkerInitializationIfLeasesExist, retryGetRecordsInSeconds, maxGetRecordsThreadPool); } @@ -1213,7 +1272,9 @@ public class Worker implements Runnable { config.getTaskBackoffTimeMillis(), config.getFailoverTimeMillis(), config.getSkipShardSyncAtWorkerInitializationIfLeasesExist(), - shardPrioritization); + shardPrioritization, + config.getRetryGetRecordsInSeconds(), + config.getMaxGetRecordsThreadPool()); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java index cbdd0a2d..d16be640 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.fail; import java.io.ByteArrayInputStream; import java.io.InputStream; +import java.util.Optional; import java.util.Set; import org.apache.commons.lang.StringUtils; @@ -60,6 +61,8 @@ public class KinesisClientLibConfiguratorTest { assertEquals(config.getApplicationName(), "b"); assertEquals(config.getStreamName(), "a"); assertEquals(config.getWorkerIdentifier(), "123"); + assertEquals(config.getMaxGetRecordsThreadPool(), Optional.empty()); + assertEquals(config.getRetryGetRecordsInSeconds(), Optional.empty()); } @Test @@ -107,7 +110,9 @@ public class KinesisClientLibConfiguratorTest { "workerId = w123", "maxRecords = 10", "metricsMaxQueueSize = 20", - "applicationName = kinesis" + "applicationName = kinesis", + "retryGetRecordsInSeconds = 2", + "maxGetRecordsThreadPool = 1" }, '\n')); assertEquals(config.getApplicationName(), "kinesis"); @@ -115,6 +120,8 @@ public class KinesisClientLibConfiguratorTest { assertEquals(config.getWorkerIdentifier(), "w123"); assertEquals(config.getMaxRecords(), 10); assertEquals(config.getMetricsMaxQueueSize(), 20); + assertEquals(config.getRetryGetRecordsInSeconds(), Optional.of(2)); + assertEquals(config.getMaxGetRecordsThreadPool(), Optional.of(1)); } @Test @@ -202,6 +209,42 @@ public class KinesisClientLibConfiguratorTest { assertEquals(config.getInitialPositionInStream(), InitialPositionInStream.TRIM_HORIZON); } + @Test + public void testEmptyOptionalVariables() { + KinesisClientLibConfiguration config = + getConfiguration(StringUtils.join(new String[] { + "streamName = a", + "applicationName = b", + "AWSCredentialsProvider = ABCD," + credentialName1, + "workerId = 123", + "initialPositionInStream = TriM_Horizon", + "maxGetRecordsThreadPool = 1" + }, '\n')); + assertEquals(config.getMaxGetRecordsThreadPool(), Optional.of(1)); + assertEquals(config.getRetryGetRecordsInSeconds(), Optional.empty()); + } + + @Test + public void testWithZeroValue() { + String test = StringUtils.join(new String[]{ + "streamName = a", + "applicationName = b", + "AWSCredentialsProvider = ABCD," + credentialName1, + "workerId = 123", + "initialPositionInStream = TriM_Horizon", + "maxGetRecordsThreadPool = 0", + "retryGetRecordsInSeconds = 0" + }, '\n'); + InputStream input = new ByteArrayInputStream(test.getBytes()); + + try { + configurator.getConfiguration(input); + } catch (Exception e) { + fail("Don't expect to fail on invalid variable value"); + + } + } + @Test public void testWithInvalidIntValue() { String test = StringUtils.join(new String[] { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java new file mode 100644 index 00000000..8518c992 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java @@ -0,0 +1,156 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { + + private static final int CORE_POOL_SIZE = 1; + private static final int MAX_POOL_SIZE = 2; + private static final int TIME_TO_LIVE = 5; + private static final int RETRY_GET_RECORDS_IN_SECONDS = 2; + private static final int SLEEP_GET_RECORDS_IN_SECONDS = 10; + + @Mock + private IKinesisProxy mockKinesisProxy; + + @Mock + private ShardInfo mockShardInfo; + + private AsynchronousGetRecordsRetrievalStrategy getRecordsRetrivalStrategy; + private KinesisDataFetcher dataFetcher; + private GetRecordsResult result; + private ExecutorService executorService; + private RejectedExecutionHandler rejectedExecutionHandler; + private int numberOfRecords = 10; + private CompletionService completionService; + + @Before + public void setup() { + dataFetcher = spy(new KinesisDataFetcherForTests(mockKinesisProxy, mockShardInfo)); + rejectedExecutionHandler = spy(new ThreadPoolExecutor.AbortPolicy()); + executorService = spy(new ThreadPoolExecutor( + CORE_POOL_SIZE, + MAX_POOL_SIZE, + TIME_TO_LIVE, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(1), + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("getrecords-worker-%d").build(), + rejectedExecutionHandler)); + getRecordsRetrivalStrategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, executorService, RETRY_GET_RECORDS_IN_SECONDS, "shardId-0001"); + completionService = spy(getRecordsRetrivalStrategy.completionService); + result = null; + } + + @Test + public void oneRequestMultithreadTest() { + GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); + verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(eq(numberOfRecords)); + verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any()); + assertNull(getRecordsResult); + } + + @Test + public void multiRequestTest() { + result = mock(GetRecordsResult.class); + + GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); + verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(numberOfRecords); + verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any()); + assertEquals(result, getRecordsResult); + + result = null; + getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); + assertNull(getRecordsResult); + } + + @Test + @Ignore + public void testInterrupted() throws InterruptedException, ExecutionException { + + Future mockFuture = mock(Future.class); + when(completionService.submit(any())).thenReturn(mockFuture); + when(completionService.poll()).thenReturn(mockFuture); + doThrow(InterruptedException.class).when(mockFuture).get(); + GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); + verify(mockFuture).get(); + assertNull(getRecordsResult); + } + + private int getLeastNumberOfCalls() { + int leastNumberOfCalls = 0; + for (int i = MAX_POOL_SIZE; i > 0; i--) { + if (i * RETRY_GET_RECORDS_IN_SECONDS <= SLEEP_GET_RECORDS_IN_SECONDS) { + leastNumberOfCalls = i; + break; + } + } + return leastNumberOfCalls; + } + + @After + public void shutdown() { + getRecordsRetrivalStrategy.shutdown(); + verify(executorService).shutdownNow(); + } + + private class KinesisDataFetcherForTests extends KinesisDataFetcher { + public KinesisDataFetcherForTests(final IKinesisProxy kinesisProxy, final ShardInfo shardInfo) { + super(kinesisProxy, shardInfo); + } + + @Override + public GetRecordsResult getRecords(final int maxRecords) { + try { + Thread.sleep(SLEEP_GET_RECORDS_IN_SECONDS * 1000); + } catch (InterruptedException e) { + // Do nothing + } + return result; + } + } + +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java new file mode 100644 index 00000000..dfba0351 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java @@ -0,0 +1,137 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.TimeUnit; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; + +/** + * + */ +@RunWith(MockitoJUnitRunner.class) +public class AsynchronousGetRecordsRetrievalStrategyTest { + + private static final long RETRY_GET_RECORDS_IN_SECONDS = 5; + private static final String SHARD_ID = "ShardId-0001"; + @Mock + private KinesisDataFetcher dataFetcher; + @Mock + private ExecutorService executorService; + @Mock + private CompletionService completionService; + @Mock + private Future successfulFuture; + @Mock + private Future blockedFuture; + @Mock + private GetRecordsResult expectedResults; + + @Test + public void testSingleSuccessfulRequestFuture() throws Exception { + AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + + when(executorService.isShutdown()).thenReturn(false); + when(completionService.submit(any())).thenReturn(successfulFuture); + when(completionService.poll(anyLong(), any())).thenReturn(successfulFuture); + when(successfulFuture.get()).thenReturn(expectedResults); + + GetRecordsResult result = strategy.getRecords(10); + + verify(executorService).isShutdown(); + verify(completionService).submit(any()); + verify(completionService).poll(eq(RETRY_GET_RECORDS_IN_SECONDS), eq(TimeUnit.SECONDS)); + verify(successfulFuture).get(); + verify(successfulFuture).cancel(eq(true)); + verify(successfulFuture).isCancelled(); + verify(completionService, never()).take(); + + assertThat(result, equalTo(expectedResults)); + } + + @Test + public void testBlockedAndSuccessfulFuture() throws Exception { + AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + + when(executorService.isShutdown()).thenReturn(false); + when(completionService.submit(any())).thenReturn(blockedFuture).thenReturn(successfulFuture); + when(completionService.poll(anyLong(), any())).thenReturn(null).thenReturn(successfulFuture); + when(successfulFuture.get()).thenReturn(expectedResults); + when(successfulFuture.cancel(anyBoolean())).thenReturn(false); + when(blockedFuture.cancel(anyBoolean())).thenReturn(true); + when(successfulFuture.isCancelled()).thenReturn(false); + when(blockedFuture.isCancelled()).thenReturn(true); + + GetRecordsResult actualResults = strategy.getRecords(10); + + verify(completionService, times(2)).submit(any()); + verify(completionService, times(2)).poll(eq(RETRY_GET_RECORDS_IN_SECONDS), eq(TimeUnit.SECONDS)); + verify(successfulFuture).get(); + verify(blockedFuture, never()).get(); + verify(successfulFuture).cancel(eq(true)); + verify(blockedFuture).cancel(eq(true)); + verify(successfulFuture).isCancelled(); + verify(blockedFuture).isCancelled(); + verify(completionService).take(); + + assertThat(actualResults, equalTo(expectedResults)); + } + + @Test(expected = IllegalStateException.class) + public void testStrategyIsShutdown() throws Exception { + AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + + when(executorService.isShutdown()).thenReturn(true); + + strategy.getRecords(10); + } + + @Test + public void testPoolOutOfResources() throws Exception { + AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + + when(executorService.isShutdown()).thenReturn(false); + when(completionService.submit(any())).thenReturn(blockedFuture).thenThrow(new RejectedExecutionException("Rejected!")).thenReturn(successfulFuture); + when(completionService.poll(anyLong(), any())).thenReturn(null).thenReturn(null).thenReturn(successfulFuture); + when(successfulFuture.get()).thenReturn(expectedResults); + when(successfulFuture.cancel(anyBoolean())).thenReturn(false); + when(blockedFuture.cancel(anyBoolean())).thenReturn(true); + when(successfulFuture.isCancelled()).thenReturn(false); + when(blockedFuture.isCancelled()).thenReturn(true); + + GetRecordsResult actualResult = strategy.getRecords(10); + + verify(completionService, times(3)).submit(any()); + verify(completionService, times(3)).poll(eq(RETRY_GET_RECORDS_IN_SECONDS), eq(TimeUnit.SECONDS)); + verify(successfulFuture).cancel(eq(true)); + verify(blockedFuture).cancel(eq(true)); + verify(successfulFuture).isCancelled(); + verify(blockedFuture).isCancelled(); + verify(completionService).take(); + + assertThat(actualResult, equalTo(expectedResults)); + } + +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java index 31272379..307aa6b8 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java @@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerStates.ConsumerState; import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerStates.ShardConsumerState; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.mockito.Mockito.never; @@ -25,6 +26,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.lang.reflect.Field; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -152,7 +154,10 @@ public class ConsumerStatesTest { } @Test - public void processingStateTest() { + public void processingStateTestSynchronous() { + when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.empty()); + when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.empty()); + ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState(); ITask task = state.createTask(consumer); @@ -163,6 +168,38 @@ public class ConsumerStatesTest { assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher))); assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig))); assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); + assertThat(task, procTask(GetRecordsRetrievalStrategy.class, "getRecordsRetrievalStrategy", instanceOf(SynchronousGetRecordsRetrievalStrategy.class) )); + + assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState())); + + assertThat(state.shutdownTransition(ShutdownReason.ZOMBIE), + equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState())); + assertThat(state.shutdownTransition(ShutdownReason.TERMINATE), + equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState())); + assertThat(state.shutdownTransition(ShutdownReason.REQUESTED), + equalTo(ShardConsumerState.SHUTDOWN_REQUESTED.getConsumerState())); + + assertThat(state.getState(), equalTo(ShardConsumerState.PROCESSING)); + assertThat(state.getTaskType(), equalTo(TaskType.PROCESS)); + + } + + @Test + public void processingStateTestAsynchronous() { + when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.of(1)); + when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.of(2)); + + ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState(); + ITask task = state.createTask(consumer); + + assertThat(task, procTask(ShardInfo.class, "shardInfo", equalTo(shardInfo))); + assertThat(task, procTask(IRecordProcessor.class, "recordProcessor", equalTo(recordProcessor))); + assertThat(task, procTask(RecordProcessorCheckpointer.class, "recordProcessorCheckpointer", + equalTo(recordProcessorCheckpointer))); + assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher))); + assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig))); + assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); + assertThat(task, procTask(GetRecordsRetrievalStrategy.class, "getRecordsRetrievalStrategy", instanceOf(AsynchronousGetRecordsRetrievalStrategy.class) )); assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState())); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java index dd56a256..2597d76b 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java @@ -117,6 +117,7 @@ public class KinesisDataFetcherTest { ICheckpoint checkpoint = mock(ICheckpoint.class); KinesisDataFetcher fetcher = new KinesisDataFetcher(kinesis, SHARD_INFO); + GetRecordsRetrievalStrategy getRecordsRetrievalStrategy = new SynchronousGetRecordsRetrievalStrategy(fetcher); String iteratorA = "foo"; String iteratorB = "bar"; @@ -138,10 +139,10 @@ public class KinesisDataFetcherTest { fetcher.initialize(seqA, null); fetcher.advanceIteratorTo(seqA, null); - Assert.assertEquals(recordsA, fetcher.getRecords(MAX_RECORDS).getRecords()); + Assert.assertEquals(recordsA, getRecordsRetrievalStrategy.getRecords(MAX_RECORDS).getRecords()); fetcher.advanceIteratorTo(seqB, null); - Assert.assertEquals(recordsB, fetcher.getRecords(MAX_RECORDS).getRecords()); + Assert.assertEquals(recordsB, getRecordsRetrievalStrategy.getRecords(MAX_RECORDS).getRecords()); } @Test @@ -181,8 +182,9 @@ public class KinesisDataFetcherTest { // Create data fectcher and initialize it with latest type checkpoint KinesisDataFetcher dataFetcher = new KinesisDataFetcher(mockProxy, SHARD_INFO); dataFetcher.initialize(SentinelCheckpoint.LATEST.toString(), INITIAL_POSITION_LATEST); + GetRecordsRetrievalStrategy getRecordsRetrievalStrategy = new SynchronousGetRecordsRetrievalStrategy(dataFetcher); // Call getRecords of dataFetcher which will throw an exception - dataFetcher.getRecords(maxRecords); + getRecordsRetrievalStrategy.getRecords(maxRecords); // Test shard has reached the end Assert.assertTrue("Shard should reach the end", dataFetcher.isShardEndReached()); @@ -206,8 +208,9 @@ public class KinesisDataFetcherTest { when(checkpoint.getCheckpoint(SHARD_ID)).thenReturn(new ExtendedSequenceNumber(seqNo)); KinesisDataFetcher fetcher = new KinesisDataFetcher(kinesis, SHARD_INFO); + GetRecordsRetrievalStrategy getRecordsRetrievalStrategy = new SynchronousGetRecordsRetrievalStrategy(fetcher); fetcher.initialize(seqNo, initialPositionInStream); - List actualRecords = fetcher.getRecords(MAX_RECORDS).getRecords(); + List actualRecords = getRecordsRetrievalStrategy.getRecords(MAX_RECORDS).getRecords(); Assert.assertEquals(expectedRecords, actualRecords); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java index e95aef50..0c47e9b9 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; @@ -76,6 +76,8 @@ public class ProcessTaskTest { private @Mock RecordProcessorCheckpointer mockCheckpointer; @Mock private ThrottlingReporter throttlingReporter; + @Mock + private GetRecordsRetrievalStrategy mockGetRecordsRetrievalStrategy; private List processedRecords; private ExtendedSequenceNumber newLargestPermittedCheckpointValue; @@ -94,19 +96,20 @@ public class ProcessTaskTest { final ShardInfo shardInfo = new ShardInfo(shardId, null, null, null); processTask = new ProcessTask( shardInfo, config, mockRecordProcessor, mockCheckpointer, mockDataFetcher, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, throttlingReporter); + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, throttlingReporter, mockGetRecordsRetrievalStrategy); } @Test public void testProcessTaskWithProvisionedThroughputExceededException() { // Set data fetcher to throw exception doReturn(false).when(mockDataFetcher).isShardEndReached(); - doThrow(new ProvisionedThroughputExceededException("Test Exception")).when(mockDataFetcher) + doThrow(new ProvisionedThroughputExceededException("Test Exception")).when(mockGetRecordsRetrievalStrategy) .getRecords(maxRecords); TaskResult result = processTask.call(); verify(throttlingReporter).throttled(); verify(throttlingReporter, never()).success(); + verify(mockGetRecordsRetrievalStrategy).getRecords(eq(maxRecords)); assertTrue("Result should contain ProvisionedThroughputExceededException", result.getException() instanceof ProvisionedThroughputExceededException); } @@ -114,9 +117,10 @@ public class ProcessTaskTest { @Test public void testProcessTaskWithNonExistentStream() { // Data fetcher returns a null Result when the stream does not exist - doReturn(null).when(mockDataFetcher).getRecords(maxRecords); + doReturn(null).when(mockGetRecordsRetrievalStrategy).getRecords(maxRecords); TaskResult result = processTask.call(); + verify(mockGetRecordsRetrievalStrategy).getRecords(eq(maxRecords)); assertNull("Task should not throw an exception", result.getException()); } @@ -300,14 +304,14 @@ public class ProcessTaskTest { private void testWithRecords(List records, ExtendedSequenceNumber lastCheckpointValue, ExtendedSequenceNumber largestPermittedCheckpointValue) { - when(mockDataFetcher.getRecords(anyInt())).thenReturn( + when(mockGetRecordsRetrievalStrategy.getRecords(anyInt())).thenReturn( new GetRecordsResult().withRecords(records)); when(mockCheckpointer.getLastCheckpointValue()).thenReturn(lastCheckpointValue); when(mockCheckpointer.getLargestPermittedCheckpointValue()).thenReturn(largestPermittedCheckpointValue); processTask.call(); verify(throttlingReporter).success(); verify(throttlingReporter, never()).throttled(); - + verify(mockGetRecordsRetrievalStrategy).getRecords(anyInt()); ArgumentCaptor priCaptor = ArgumentCaptor.forClass(ProcessRecordsInput.class); verify(mockRecordProcessor).processRecords(priCaptor.capture()); processedRecords = priCaptor.getValue().getRecords(); From 01d2688bc6e68fd3fe5cb698cb65299d67ac930d Mon Sep 17 00:00:00 2001 From: Alex Charlton Date: Tue, 19 Sep 2017 08:52:31 -0700 Subject: [PATCH 05/35] MultiLangDaemon: Make shutdown grace configurable (#204) Allow configuring the amount of time that the graceful shutdown process will wait for the client to complete its shutdown. --- .../worker/KinesisClientLibConfiguration.java | 37 +++++++++++++++++-- .../kinesis/multilang/MultiLangDaemon.java | 3 +- .../KinesisClientLibConfigurationTest.java | 14 ++++--- 3 files changed, 44 insertions(+), 10 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 62d87f30..1bfd0fc0 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -172,6 +172,11 @@ public class KinesisClientLibConfiguration { */ public static final ShardPrioritization DEFAULT_SHARD_PRIORITIZATION = new NoOpShardPrioritization(); + /** + * The amount of milliseconds to wait before graceful shutdown forcefully terminates. + */ + public static final long DEFAULT_SHUTDOWN_GRACE_MILLIS = 5000L; + /** * The size of the thread pool to create for the lease renewer to use. */ @@ -213,6 +218,7 @@ public class KinesisClientLibConfiguration { // This is useful for optimizing deployments to large fleets working on a stable stream. private boolean skipShardSyncAtWorkerInitializationIfLeasesExist; private ShardPrioritization shardPrioritization; + private long shutdownGraceMillis; @Getter private Optional timeoutInSeconds = Optional.empty(); @@ -268,7 +274,8 @@ public class KinesisClientLibConfiguration { DEFAULT_SHARD_SYNC_INTERVAL_MILLIS, DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION, new ClientConfiguration(), new ClientConfiguration(), new ClientConfiguration(), DEFAULT_TASK_BACKOFF_TIME_MILLIS, DEFAULT_METRICS_BUFFER_TIME_MILLIS, DEFAULT_METRICS_MAX_QUEUE_SIZE, - DEFAULT_VALIDATE_SEQUENCE_NUMBER_BEFORE_CHECKPOINTING, null); + DEFAULT_VALIDATE_SEQUENCE_NUMBER_BEFORE_CHECKPOINTING, null, + DEFAULT_SHUTDOWN_GRACE_MILLIS); } /** @@ -303,6 +310,7 @@ public class KinesisClientLibConfiguration { * with a call to Amazon Kinesis before checkpointing for calls to * {@link RecordProcessorCheckpointer#checkpoint(String)} * @param regionName The region name for the service + * @param shutdownGraceMillis The number of milliseconds before graceful shutdown terminates forcefully */ // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES @@ -328,7 +336,8 @@ public class KinesisClientLibConfiguration { long metricsBufferTimeMillis, int metricsMaxQueueSize, boolean validateSequenceNumberBeforeCheckpointing, - String regionName) { + String regionName, + long shutdownGraceMillis) { this(applicationName, streamName, kinesisEndpoint, null, initialPositionInStream, kinesisCredentialsProvider, dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId, maxRecords, idleTimeBetweenReadsInMillis, @@ -336,7 +345,7 @@ public class KinesisClientLibConfiguration { shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry, kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig, taskBackoffTimeMillis, metricsBufferTimeMillis, metricsMaxQueueSize, - validateSequenceNumberBeforeCheckpointing, regionName); + validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis); } /** @@ -398,7 +407,8 @@ public class KinesisClientLibConfiguration { long metricsBufferTimeMillis, int metricsMaxQueueSize, boolean validateSequenceNumberBeforeCheckpointing, - String regionName) { + String regionName, + long shutdownGraceMillis) { // Check following values are greater than zero checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis); checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis); @@ -408,6 +418,7 @@ public class KinesisClientLibConfiguration { checkIsValuePositive("TaskBackoffTimeMillis", taskBackoffTimeMillis); checkIsValuePositive("MetricsBufferTimeMills", metricsBufferTimeMillis); checkIsValuePositive("MetricsMaxQueueSize", (long) metricsMaxQueueSize); + checkIsValuePositive("ShutdownGraceMillis", shutdownGraceMillis); checkIsRegionNameValid(regionName); this.applicationName = applicationName; this.tableName = applicationName; @@ -444,6 +455,7 @@ public class KinesisClientLibConfiguration { InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; + this.shutdownGraceMillis = shutdownGraceMillis; } // Check if value is positive, otherwise throw an exception @@ -731,6 +743,14 @@ public class KinesisClientLibConfiguration { return shardPrioritization; } + /** + * @return Graceful shutdown timeout + */ + public long getShutdownGraceMillis() { + return shutdownGraceMillis; + } + + /* // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 190 LINES /** * @param tableName name of the lease table in DynamoDB @@ -1145,4 +1165,13 @@ public class KinesisClientLibConfiguration { this.timeoutInSeconds = Optional.of(timeoutInSeconds); } + /** + * @param shutdownGraceMillis Time before gracefully shutdown forcefully terminates + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withShutdownGraceMillis(long shutdownGraceMillis) { + checkIsValuePositive("ShutdownGraceMillis", shutdownGraceMillis); + this.shutdownGraceMillis = shutdownGraceMillis; + return this; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java index 885eeb4f..82ca74c1 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java @@ -148,13 +148,14 @@ public class MultiLangDaemon implements Callable { config.getRecordProcessorFactory(), executorService); + final long shutdownGraceMillis = config.getKinesisClientLibConfiguration().getShutdownGraceMillis(); Runtime.getRuntime().addShutdownHook(new Thread() { @Override public void run() { LOG.info("Process terminanted, will initiate shutdown."); try { Future fut = daemon.worker.requestShutdown(); - fut.get(5000, TimeUnit.MILLISECONDS); + fut.get(shutdownGraceMillis, TimeUnit.MILLISECONDS); LOG.info("Process shutdown is complete."); } catch (InterruptedException | ExecutionException | TimeoutException e) { LOG.error("Encountered an error during shutdown.", e); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java index 4874a164..0092699f 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java @@ -84,7 +84,8 @@ public class KinesisClientLibConfigurationTest { TEST_VALUE_LONG, TEST_VALUE_INT, skipCheckpointValidationValue, - null); + null, + TEST_VALUE_LONG); } @Test @@ -94,7 +95,7 @@ public class KinesisClientLibConfigurationTest { // Try each argument at one time. KinesisClientLibConfiguration config = null; long[] longValues = - { TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG }; + { TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG }; for (int i = 0; i < PARAMETER_COUNT; i++) { longValues[i] = INVALID_LONG; try { @@ -122,7 +123,8 @@ public class KinesisClientLibConfigurationTest { longValues[5], TEST_VALUE_INT, skipCheckpointValidationValue, - null); + null, + longValues[6]); } catch (IllegalArgumentException e) { System.out.println(e.getMessage()); } @@ -156,7 +158,8 @@ public class KinesisClientLibConfigurationTest { TEST_VALUE_LONG, intValues[1], skipCheckpointValidationValue, - null); + null, + TEST_VALUE_LONG); } catch (IllegalArgumentException e) { System.out.println(e.getMessage()); } @@ -319,7 +322,8 @@ public class KinesisClientLibConfigurationTest { TEST_VALUE_LONG, 1, skipCheckpointValidationValue, - "abcd"); + "abcd", + TEST_VALUE_LONG); Assert.fail("No expected Exception is thrown."); } catch(IllegalArgumentException e) { System.out.println(e.getMessage()); From 9a82b6bd05b3c9c5f8581af007141fa6d5f0fc4e Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Wed, 20 Sep 2017 08:52:36 -0700 Subject: [PATCH 06/35] Release 1.8.2 of the Amazon Kinesis Client for Java (#218) * Add support for two phase checkpoints Applications can now set a pending checkpoint, before completing the checkpoint operation. Once the application has completed its checkpoint steps, the final checkpoint will clear the pending checkpoint. Should the checkpoint fail the attempted sequence number is provided in the InitializationInput#getPendingCheckpointSequenceNumber otherwise the value will be null. * PR #188 * Support timeouts, and retry for GetRecords calls. Applications can now set timeouts for GetRecord calls to Kinesis. As part of setting the timeout, the application must also provide a thread pool size for concurrent requests. * PR #214 * Notification when the lease table is throttled When writes, or reads, to the lease table are throttled a warning will be emitted. If you're seeing this warning you should increase the IOPs for your lease table to prevent processing delays. * PR #212 * Support configuring the graceful shutdown timeout for MultiLang Clients This adds support for setting the timeout that the Java process will wait for the MutliLang client to complete graceful shutdown. The timeout can be configured by adding shutdownGraceMillis to the properties file set to the number of milliseconds to wait. * PR #204 --- README.md | 15 +++++++++++++++ pom.xml | 4 ++-- ...synchronousGetRecordsRetrievalStrategy.java | 14 ++++++++++++++ .../DoesNothingPreparedCheckpointer.java | 14 ++++++++++++++ .../worker/GetRecordsRetrievalStrategy.java | 14 ++++++++++++++ .../lib/worker/PreparedCheckpointer.java | 14 ++++++++++++++ .../clientlibrary/lib/worker/ProcessTask.java | 18 +++++++++--------- .../worker/RecordProcessorCheckpointer.java | 2 +- .../lib/worker/ShardConsumer.java | 18 +++++++++--------- .../clientlibrary/lib/worker/ShardInfo.java | 18 +++++++++--------- .../lib/worker/ShutdownReason.java | 18 +++++++++--------- ...SynchronousGetRecordsRetrievalStrategy.java | 14 ++++++++++++++ .../clientlibrary/proxies/KinesisProxy.java | 18 +++++++++--------- .../types/InitializationInput.java | 18 +++++++++--------- .../types/ProcessRecordsInput.java | 18 +++++++++--------- .../clientlibrary/types/ShutdownInput.java | 18 +++++++++--------- .../leases/impl/KinesisClientLease.java | 18 +++++++++--------- .../impl/KinesisClientLeaseSerializer.java | 18 +++++++++--------- .../kinesis/leases/impl/LeaseManager.java | 18 +++++++++--------- .../kinesis/multilang/MultiLangDaemon.java | 18 +++++++++--------- .../multilang/MultiLangDaemonConfig.java | 18 +++++++++--------- .../kinesis/multilang/MultiLangProtocol.java | 18 +++++++++--------- .../multilang/MultiLangRecordProcessor.java | 18 +++++++++--------- .../MultiLangRecordProcessorFactory.java | 18 +++++++++--------- .../multilang/messages/CheckpointMessage.java | 18 +++++++++--------- .../multilang/messages/InitializeMessage.java | 18 +++++++++--------- .../multilang/messages/JsonFriendlyRecord.java | 18 +++++++++--------- .../kinesis/multilang/messages/Message.java | 18 +++++++++--------- .../messages/ProcessRecordsMessage.java | 18 +++++++++--------- .../messages/ShutdownRequestedMessage.java | 14 ++++++++++++++ .../KinesisClientLibConfiguratorTest.java | 18 +++++++++--------- ...hronousGetRecordsRetrievalStrategyTest.java | 14 ++++++++++++++ .../KinesisClientLibConfigurationTest.java | 18 +++++++++--------- .../lib/worker/KinesisDataFetcherTest.java | 18 +++++++++--------- .../lib/worker/ProcessTaskTest.java | 18 +++++++++--------- .../kinesis/multilang/MessageWriterTest.java | 18 +++++++++--------- .../multilang/MultiLangProtocolTest.java | 18 +++++++++--------- .../StreamingRecordProcessorFactoryTest.java | 18 +++++++++--------- .../StreamingRecordProcessorTest.java | 18 +++++++++--------- .../multilang/messages/MessageTest.java | 18 +++++++++--------- 40 files changed, 386 insertions(+), 273 deletions(-) diff --git a/README.md b/README.md index e0320dbe..fa566946 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,21 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.2 (September 20, 2017) +* Add support for two phase checkpoints + Applications can now set a pending checkpoint, before completing the checkpoint operation. Once the application has completed its checkpoint steps, the final checkpoint will clear the pending checkpoint. + Should the checkpoint fail the attempted sequence number is provided in the [`InitializationInput#getPendingCheckpointSequenceNumber`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java#L81) otherwise the value will be null. + * [PR #188](https://github.com/awslabs/amazon-kinesis-client/pull/188) +* Support timeouts, and retry for GetRecords calls. + Applications can now set timeouts for GetRecord calls to Kinesis. As part of setting the timeout, the application must also provide a thread pool size for concurrent requests. + * [PR #214](https://github.com/awslabs/amazon-kinesis-client/pulls/214) +* Notification when the lease table is throttled + When writes, or reads, to the lease table are throttled a warning will be emitted. If you're seeing this warning you should increase the IOPs for your lease table to prevent processing delays. + * [PR #212](https://github.com/awslabs/amazon-kinesis-client/pulls/212) +* Support configuring the graceful shutdown timeout for MultiLang Clients + This adds support for setting the timeout that the Java process will wait for the MutliLang client to complete graceful shutdown. The timeout can be configured by adding `shutdownGraceMillis` to the properties file set to the number of milliseconds to wait. + * [PR #204](https://github.com/awslabs/amazon-kinesis-client/pull/204) + ### Release 1.8.1 (August 2, 2017) * Support timeouts for calls to the MultiLang Daemon This adds support for setting a timeout when dispatching records to the client record processor. If the record processor doesn't respond within the timeout the parent Java process will be terminated. This is a temporary fix to handle cases where the KCL becomes blocked while waiting for a client record processor. diff --git a/pom.xml b/pom.xml index 981f9d70..e34edaec 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.2-SNAPSHOT + 1.8.2 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. @@ -25,7 +25,7 @@ - 1.11.171 + 1.11.198 1.0.392 libsqlite4java ${project.build.directory}/test-lib diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java index 6290dd4f..92057327 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.HashSet; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java index ed72f317..d40f51d3 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DoesNothingPreparedCheckpointer.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java index a391ac59..8f7afe25 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import com.amazonaws.services.kinesis.model.GetRecordsResult; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointer.java index 1b399fcc..b7b4ba9d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PreparedCheckpointer.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java index 223236f6..02fc4d70 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java index 25a106f7..72e18d73 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java @@ -1,5 +1,5 @@ /* - * Copyright 2012-2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * * Licensed under the Amazon Software License (the "License"). * You may not use this file except in compliance with the License. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 70a81fbc..69057b38 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java index c339e9f9..e681d905 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardInfo.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2013 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownReason.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownReason.java index 8d0dfc80..05925120 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownReason.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownReason.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2013 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java index 77a60448..3c8925b0 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import com.amazonaws.services.kinesis.model.GetRecordsResult; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java index b60fc938..fd45c764 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/proxies/KinesisProxy.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.proxies; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java index a44aa844..fce165f2 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/InitializationInput.java @@ -1,16 +1,16 @@ /* - * Copyright 2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.types; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java index f617e5e8..bd960c08 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java @@ -1,16 +1,16 @@ /* - * Copyright 2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.types; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ShutdownInput.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ShutdownInput.java index c533a4da..368dd3d3 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ShutdownInput.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ShutdownInput.java @@ -1,16 +1,16 @@ /* - * Copyright 2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.types; diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java index f3b2e828..5f2d56b0 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLease.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2013 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java index 2383ff10..1234e164 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/KinesisClientLeaseSerializer.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index 6a70eb90..a2bf33a2 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java index 82ca74c1..2c8d6909 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemon.java @@ -1,16 +1,16 @@ /* - * Copyright 2014-2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemonConfig.java b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemonConfig.java index 01b3a27f..fc143083 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemonConfig.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangDaemonConfig.java @@ -1,16 +1,16 @@ /* - * Copyright 2014-2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocol.java b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocol.java index 0376242a..7a809289 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocol.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocol.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessor.java b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessor.java index bbcf957b..1261c06a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessor.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessor.java @@ -1,16 +1,16 @@ /* - * Copyright 2014-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessorFactory.java b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessorFactory.java index e596abf2..eadb1f6d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessorFactory.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/MultiLangRecordProcessorFactory.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/CheckpointMessage.java b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/CheckpointMessage.java index 5cdc02bd..f38980ba 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/CheckpointMessage.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/CheckpointMessage.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang.messages; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/InitializeMessage.java b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/InitializeMessage.java index 3795e57e..cc6be56f 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/InitializeMessage.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/InitializeMessage.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang.messages; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/JsonFriendlyRecord.java b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/JsonFriendlyRecord.java index 600489fe..19100993 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/JsonFriendlyRecord.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/JsonFriendlyRecord.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang.messages; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/Message.java b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/Message.java index 48ad09d0..7470b8e2 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/Message.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/Message.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang.messages; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ProcessRecordsMessage.java b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ProcessRecordsMessage.java index 9e382b93..12371eb8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ProcessRecordsMessage.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ProcessRecordsMessage.java @@ -1,16 +1,16 @@ /* - * Copyright 2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang.messages; diff --git a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ShutdownRequestedMessage.java b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ShutdownRequestedMessage.java index 6cf77964..409cbce4 100644 --- a/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ShutdownRequestedMessage.java +++ b/src/main/java/com/amazonaws/services/kinesis/multilang/messages/ShutdownRequestedMessage.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.multilang.messages; /** diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java index d16be640..72f171fc 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/config/KinesisClientLibConfiguratorTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.config; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java index dfba0351..9ecea68d 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java @@ -1,3 +1,17 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import static org.hamcrest.CoreMatchers.equalTo; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java index 0092699f..cfa8be10 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java index 2597d76b..2b89c3c8 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java index 0c47e9b9..b24bf3ec 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; diff --git a/src/test/java/com/amazonaws/services/kinesis/multilang/MessageWriterTest.java b/src/test/java/com/amazonaws/services/kinesis/multilang/MessageWriterTest.java index 21771980..f9fd1d58 100644 --- a/src/test/java/com/amazonaws/services/kinesis/multilang/MessageWriterTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/multilang/MessageWriterTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/test/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocolTest.java b/src/test/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocolTest.java index 3f35b8fa..da14d256 100644 --- a/src/test/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocolTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/multilang/MultiLangProtocolTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorFactoryTest.java b/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorFactoryTest.java index aa6aceea..ba3e735b 100644 --- a/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorFactoryTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorFactoryTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java b/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java index d2659349..f32fa5bf 100644 --- a/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/multilang/StreamingRecordProcessorTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang; diff --git a/src/test/java/com/amazonaws/services/kinesis/multilang/messages/MessageTest.java b/src/test/java/com/amazonaws/services/kinesis/multilang/messages/MessageTest.java index e9a976ae..2b2fe402 100644 --- a/src/test/java/com/amazonaws/services/kinesis/multilang/messages/MessageTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/multilang/messages/MessageTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.multilang.messages; From 4dd9423170c8444d6f31cf0add77556f09df491b Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Fri, 22 Sep 2017 11:43:34 -0700 Subject: [PATCH 07/35] Calling shutdown on the RetrievalStrategy (#222) Fixes a bug where the retriever wasn't being shutdown when a record processor was being shutdown. --- README.md | 4 +- pom.xml | 2 +- .../lib/worker/ConsumerStates.java | 5 +- .../clientlibrary/lib/worker/ProcessTask.java | 42 +--------- .../lib/worker/ShardConsumer.java | 21 +++-- .../lib/worker/ShutdownTask.java | 41 +++++----- .../lib/worker/ConsumerStatesTest.java | 14 +--- .../lib/worker/ShardConsumerTest.java | 76 ++++++++++++++++--- .../lib/worker/ShutdownTaskTest.java | 37 ++++++--- 9 files changed, 143 insertions(+), 99 deletions(-) diff --git a/README.md b/README.md index fa566946..2f2c0fd6 100644 --- a/README.md +++ b/README.md @@ -36,10 +36,10 @@ To make it easier for developers to write record processors in other languages, * [PR #188](https://github.com/awslabs/amazon-kinesis-client/pull/188) * Support timeouts, and retry for GetRecords calls. Applications can now set timeouts for GetRecord calls to Kinesis. As part of setting the timeout, the application must also provide a thread pool size for concurrent requests. - * [PR #214](https://github.com/awslabs/amazon-kinesis-client/pulls/214) + * [PR #214](https://github.com/awslabs/amazon-kinesis-client/pull/214) * Notification when the lease table is throttled When writes, or reads, to the lease table are throttled a warning will be emitted. If you're seeing this warning you should increase the IOPs for your lease table to prevent processing delays. - * [PR #212](https://github.com/awslabs/amazon-kinesis-client/pulls/212) + * [PR #212](https://github.com/awslabs/amazon-kinesis-client/pull/212) * Support configuring the graceful shutdown timeout for MultiLang Clients This adds support for setting the timeout that the Java process will wait for the MutliLang client to complete graceful shutdown. The timeout can be configured by adding `shutdownGraceMillis` to the properties file set to the number of milliseconds to wait. * [PR #204](https://github.com/awslabs/amazon-kinesis-client/pull/204) diff --git a/pom.xml b/pom.xml index e34edaec..4c281f20 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.2 + 1.8.3-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index f6d96b4d..d3ccb911 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -312,7 +312,7 @@ class ConsumerStates { return new ProcessTask(consumer.getShardInfo(), consumer.getStreamConfig(), consumer.getRecordProcessor(), consumer.getRecordProcessorCheckpointer(), consumer.getDataFetcher(), consumer.getTaskBackoffTimeMillis(), consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist(), - consumer.getRetryGetRecordsInSeconds(), consumer.getMaxGetRecordsThreadPool()); + consumer.getGetRecordsRetrievalStrategy()); } @Override @@ -516,7 +516,8 @@ class ConsumerStates { consumer.getStreamConfig().getStreamProxy(), consumer.getStreamConfig().getInitialPositionInStream(), consumer.isCleanupLeasesOfCompletedShards(), consumer.getLeaseManager(), - consumer.getTaskBackoffTimeMillis()); + consumer.getTaskBackoffTimeMillis(), + consumer.getGetRecordsRetrievalStrategy()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java index 02fc4d70..90ac2c09 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java @@ -65,17 +65,6 @@ class ProcessTask implements ITask { private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; - private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher, - Optional retryGetRecordsInSeconds, - Optional maxGetRecordsThreadPool, - ShardInfo shardInfo) { - Optional getRecordsRetrievalStrategy = retryGetRecordsInSeconds.flatMap(retry -> - maxGetRecordsThreadPool.map(max -> - new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, retry, max, shardInfo.getShardId()))); - - return getRecordsRetrievalStrategy.orElse(new SynchronousGetRecordsRetrievalStrategy(dataFetcher)); - } - /** * @param shardInfo * contains information about the shard @@ -89,40 +78,17 @@ class ProcessTask implements ITask { * Kinesis data fetcher (used to fetch records from Kinesis) * @param backoffTimeMillis * backoff time when catching exceptions - */ - public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, - RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, - long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist) { - this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis, - skipShardSyncAtWorkerInitializationIfLeasesExist, Optional.empty(), Optional.empty()); - } - - /** - * @param shardInfo - * contains information about the shard - * @param streamConfig - * Stream configuration - * @param recordProcessor - * Record processor used to process the data records for the shard - * @param recordProcessorCheckpointer - * Passed to the RecordProcessor so it can checkpoint progress - * @param dataFetcher - * Kinesis data fetcher (used to fetch records from Kinesis) - * @param backoffTimeMillis - * backoff time when catching exceptions - * @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 getRecordsRetrievalStrategy + * The retrieval strategy for fetching records from kinesis */ public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, - Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool) { + GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()), - makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo)); + getRecordsRetrievalStrategy); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 69057b38..4bbe1939 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -55,14 +55,24 @@ class ShardConsumer { private final boolean cleanupLeasesOfCompletedShards; private final long taskBackoffTimeMillis; private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist; - @Getter - private final Optional retryGetRecordsInSeconds; - @Getter - private final Optional maxGetRecordsThreadPool; private ITask currentTask; private long currentTaskSubmitTime; private Future future; + + @Getter + private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + + private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher, + Optional retryGetRecordsInSeconds, + Optional maxGetRecordsThreadPool, + ShardInfo shardInfo) { + Optional getRecordsRetrievalStrategy = retryGetRecordsInSeconds.flatMap(retry -> + maxGetRecordsThreadPool.map(max -> + new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, retry, max, shardInfo.getShardId()))); + + return getRecordsRetrievalStrategy.orElse(new SynchronousGetRecordsRetrievalStrategy(dataFetcher)); + } /* * Tracks current state. It is only updated via the consumeStream/shutdown APIs. Therefore we don't do @@ -149,8 +159,7 @@ class ShardConsumer { this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards; this.taskBackoffTimeMillis = backoffTimeMillis; this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist; - this.retryGetRecordsInSeconds = retryGetRecordsInSeconds; - this.maxGetRecordsThreadPool = maxGetRecordsThreadPool; + this.getRecordsRetrievalStrategy = makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index d40fbb0e..f56033a8 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; @@ -46,20 +46,22 @@ class ShutdownTask implements ITask { private final boolean cleanupLeasesOfCompletedShards; private final TaskType taskType = TaskType.SHUTDOWN; private final long backoffTimeMillis; + private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; /** * Constructor. */ // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES ShutdownTask(ShardInfo shardInfo, - IRecordProcessor recordProcessor, - RecordProcessorCheckpointer recordProcessorCheckpointer, - ShutdownReason reason, - IKinesisProxy kinesisProxy, - InitialPositionInStreamExtended initialPositionInStream, - boolean cleanupLeasesOfCompletedShards, - ILeaseManager leaseManager, - long backoffTimeMillis) { + IRecordProcessor recordProcessor, + RecordProcessorCheckpointer recordProcessorCheckpointer, + ShutdownReason reason, + IKinesisProxy kinesisProxy, + InitialPositionInStreamExtended initialPositionInStream, + boolean cleanupLeasesOfCompletedShards, + ILeaseManager leaseManager, + long backoffTimeMillis, + GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -69,6 +71,7 @@ class ShutdownTask implements ITask { this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards; this.leaseManager = leaseManager; this.backoffTimeMillis = backoffTimeMillis; + this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; } /* @@ -79,7 +82,7 @@ class ShutdownTask implements ITask { */ @Override public TaskResult call() { - Exception exception = null; + Exception exception; boolean applicationException = false; try { @@ -107,6 +110,8 @@ class ShutdownTask implements ITask { + shardInfo.getShardId()); } } + LOG.debug("Shutting down retrieval strategy."); + getRecordsRetrievalStrategy.shutdown(); LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId()); } catch (Exception e) { applicationException = true; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java index 307aa6b8..63f20a72 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java @@ -17,7 +17,6 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerStates.ConsumerState; import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerStates.ShardConsumerState; import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.mockito.Mockito.never; @@ -26,7 +25,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.lang.reflect.Field; -import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -76,6 +74,8 @@ public class ConsumerStatesTest { private IKinesisProxy kinesisProxy; @Mock private InitialPositionInStreamExtended initialPositionInStream; + @Mock + private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private long parentShardPollIntervalMillis = 0xCAFE; private boolean cleanupLeasesOfCompletedShards = true; @@ -98,7 +98,7 @@ public class ConsumerStatesTest { when(consumer.isCleanupLeasesOfCompletedShards()).thenReturn(cleanupLeasesOfCompletedShards); when(consumer.getTaskBackoffTimeMillis()).thenReturn(taskBackoffTimeMillis); when(consumer.getShutdownReason()).thenReturn(reason); - + when(consumer.getGetRecordsRetrievalStrategy()).thenReturn(getRecordsRetrievalStrategy); } private static final Class> LEASE_MANAGER_CLASS = (Class>) (Class) ILeaseManager.class; @@ -155,9 +155,6 @@ public class ConsumerStatesTest { @Test public void processingStateTestSynchronous() { - when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.empty()); - when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.empty()); - ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState(); ITask task = state.createTask(consumer); @@ -168,7 +165,6 @@ public class ConsumerStatesTest { assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher))); assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig))); assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); - assertThat(task, procTask(GetRecordsRetrievalStrategy.class, "getRecordsRetrievalStrategy", instanceOf(SynchronousGetRecordsRetrievalStrategy.class) )); assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState())); @@ -186,9 +182,6 @@ public class ConsumerStatesTest { @Test public void processingStateTestAsynchronous() { - when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.of(1)); - when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.of(2)); - ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState(); ITask task = state.createTask(consumer); @@ -199,7 +192,6 @@ public class ConsumerStatesTest { assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher))); assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig))); assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); - assertThat(task, procTask(GetRecordsRetrievalStrategy.class, "getRecordsRetrievalStrategy", instanceOf(AsynchronousGetRecordsRetrievalStrategy.class) )); assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState())); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 8073d0df..a3f786a6 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; @@ -39,6 +40,7 @@ import java.util.Date; import java.util.List; import java.util.ListIterator; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -509,6 +511,62 @@ public class ShardConsumerTest { Thread.sleep(50L); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING))); } + + @Test + public void testCreateSynchronousGetRecordsRetrieval() { + ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + StreamConfig streamConfig = + new StreamConfig(streamProxy, + 1, + 10, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + ShardConsumer shardConsumer = + new ShardConsumer(shardInfo, + streamConfig, + checkpoint, + processor, + null, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + Optional.empty(), + Optional.empty()); + + assertEquals(shardConsumer.getGetRecordsRetrievalStrategy().getClass(), SynchronousGetRecordsRetrievalStrategy.class); + } + + @Test + public void testCreateAsynchronousGetRecordsRetrieval() { + ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.TRIM_HORIZON); + StreamConfig streamConfig = + new StreamConfig(streamProxy, + 1, + 10, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + ShardConsumer shardConsumer = + new ShardConsumer(shardInfo, + streamConfig, + checkpoint, + processor, + null, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + Optional.of(1), + Optional.of(2)); + + assertEquals(shardConsumer.getGetRecordsRetrievalStrategy().getClass(), AsynchronousGetRecordsRetrievalStrategy.class); + } //@formatter:off (gets the formatting wrong) private void verifyConsumedRecords(List expectedRecords, diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 9eaf7e8e..5d91c698 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -1,20 +1,22 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.HashSet; @@ -34,10 +36,14 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLeaseManager; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; /** * */ +@RunWith(MockitoJUnitRunner.class) public class ShutdownTaskTest { private static final long TASK_BACKOFF_TIME_MILLIS = 1L; private static final InitialPositionInStreamExtended INITIAL_POSITION_TRIM_HORIZON = @@ -51,6 +57,9 @@ public class ShutdownTaskTest { defaultParentShardIds, ExtendedSequenceNumber.LATEST); IRecordProcessor defaultRecordProcessor = new TestStreamlet(); + + @Mock + private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; /** * @throws java.lang.Exception @@ -71,6 +80,7 @@ public class ShutdownTaskTest { */ @Before public void setUp() throws Exception { + doNothing().when(getRecordsRetrievalStrategy).shutdown(); } /** @@ -98,7 +108,8 @@ public class ShutdownTaskTest { INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, leaseManager, - TASK_BACKOFF_TIME_MILLIS); + TASK_BACKOFF_TIME_MILLIS, + getRecordsRetrievalStrategy); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof IllegalArgumentException); @@ -123,10 +134,12 @@ public class ShutdownTaskTest { INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, leaseManager, - TASK_BACKOFF_TIME_MILLIS); + TASK_BACKOFF_TIME_MILLIS, + getRecordsRetrievalStrategy); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof KinesisClientLibIOException); + verify(getRecordsRetrievalStrategy).shutdown(); } /** @@ -134,7 +147,7 @@ public class ShutdownTaskTest { */ @Test public final void testGetTaskType() { - ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, null, 0); + ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, null, 0, getRecordsRetrievalStrategy); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } From 55584490436a4ba9d42e58db2eb633c48722aae0 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Fri, 22 Sep 2017 12:25:52 -0700 Subject: [PATCH 08/35] Release 1.8.3 of the Amazon Kinesis Client for Java (#224) * Call shutdown on the retriever when the record processor is being shutdown This fixes a bug that could leak threads if using the AsynchronousGetRecordsRetrievalStrategy is being used. The asynchronous retriever is only used when KinesisClientLibConfiguration#retryGetRecordsInSeconds, and KinesisClientLibConfiguration#maxGetRecordsThreadPool are set. * PR #222 --- META-INF/MANIFEST.MF | 2 +- README.md | 6 ++++++ pom.xml | 2 +- .../lib/worker/KinesisClientLibConfiguration.java | 2 +- 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index 9665aebd..3a8282e4 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.2 +Bundle-Version: 1.8.3 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.7 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/README.md b/README.md index 2f2c0fd6..ddaa6194 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,12 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.3 (September 22, 2017) +* Call shutdown on the retriever when the record processor is being shutdown + This fixes a bug that could leak threads if using the [`AsynchronousGetRecordsRetrievalStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/9a82b6bd05b3c9c5f8581af007141fa6d5f0fc4e/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java#L42) is being used. + The asynchronous retriever is only used when [`KinesisClientLibConfiguration#retryGetRecordsInSeconds`](https://github.com/awslabs/amazon-kinesis-client/blob/01d2688bc6e68fd3fe5cb698cb65299d67ac930d/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L227), and [`KinesisClientLibConfiguration#maxGetRecordsThreadPool`](https://github.com/awslabs/amazon-kinesis-client/blob/01d2688bc6e68fd3fe5cb698cb65299d67ac930d/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L230) are set. + * [PR #222](https://github.com/awslabs/amazon-kinesis-client/pull/222) + ### Release 1.8.2 (September 20, 2017) * Add support for two phase checkpoints Applications can now set a pending checkpoint, before completing the checkpoint operation. Once the application has completed its checkpoint steps, the final checkpoint will clear the pending checkpoint. diff --git a/pom.xml b/pom.xml index 4c281f20..61c8c6cf 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.3-SNAPSHOT + 1.8.3 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 1bfd0fc0..c970daa0 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.2"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.3"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From b1c27bd386d5c09797406a639638b2561d807e6c Mon Sep 17 00:00:00 2001 From: "Pfifer, Justin" Date: Fri, 22 Sep 2017 14:45:54 -0700 Subject: [PATCH 09/35] Recreate the completion service instead of reusing them. This prevents cancelled tasks from being polled in subsequent calls. --- META-INF/MANIFEST.MF | 2 +- pom.xml | 2 +- ...ynchronousGetRecordsRetrievalStrategy.java | 20 +++++++------- .../worker/KinesisClientLibConfiguration.java | 2 +- ...cordsRetrievalStrategyIntegrationTest.java | 17 +++++++++--- ...ronousGetRecordsRetrievalStrategyTest.java | 26 ++++++++++--------- 6 files changed, 40 insertions(+), 29 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index 3a8282e4..a7ac9a5a 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.3 +Bundle-Version: 1.8.4 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.7 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/pom.xml b/pom.xml index 61c8c6cf..5d8bdb53 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.3 + 1.8.4-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java index 92057327..b592c29b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java @@ -16,6 +16,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; @@ -26,6 +27,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingScope; @@ -47,7 +49,7 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie private final ExecutorService executorService; private final int retryGetRecordsInSeconds; private final String shardId; - final CompletionService completionService; + final Supplier> completionServiceSupplier; public AsynchronousGetRecordsRetrievalStrategy(@NonNull final KinesisDataFetcher dataFetcher, final int retryGetRecordsInSeconds, final int maxGetRecordsThreadPool, String shardId) { @@ -56,16 +58,17 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie public AsynchronousGetRecordsRetrievalStrategy(final KinesisDataFetcher dataFetcher, final ExecutorService executorService, final int retryGetRecordsInSeconds, String shardId) { - this(dataFetcher, executorService, retryGetRecordsInSeconds, new ExecutorCompletionService<>(executorService), + this(dataFetcher, executorService, retryGetRecordsInSeconds, () -> new ExecutorCompletionService<>(executorService), shardId); } AsynchronousGetRecordsRetrievalStrategy(KinesisDataFetcher dataFetcher, ExecutorService executorService, - int retryGetRecordsInSeconds, CompletionService completionService, String shardId) { + int retryGetRecordsInSeconds, Supplier> completionServiceSupplier, + String shardId) { this.dataFetcher = dataFetcher; this.executorService = executorService; this.retryGetRecordsInSeconds = retryGetRecordsInSeconds; - this.completionService = completionService; + this.completionServiceSupplier = completionServiceSupplier; this.shardId = shardId; } @@ -75,6 +78,7 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie throw new IllegalStateException("Strategy has been shutdown"); } GetRecordsResult result = null; + CompletionService completionService = completionServiceSupplier.get(); Set> futures = new HashSet<>(); Callable retrieverCall = createRetrieverCallable(maxRecords); while (true) { @@ -98,13 +102,7 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie break; } } - futures.stream().peek(f -> f.cancel(true)).filter(Future::isCancelled).forEach(f -> { - try { - completionService.take(); - } catch (InterruptedException e) { - log.error("Exception thrown while trying to empty the threadpool."); - } - }); + futures.forEach(f -> f.cancel(true)); return result; } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index c970daa0..4620bfe4 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.3"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.4"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java index 8518c992..b3659605 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java @@ -27,12 +27,14 @@ import org.mockito.runners.MockitoJUnitRunner; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -59,6 +61,10 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { @Mock private ShardInfo mockShardInfo; + @Mock + private Supplier> completionServiceSupplier; + + private CompletionService completionService; private AsynchronousGetRecordsRetrievalStrategy getRecordsRetrivalStrategy; private KinesisDataFetcher dataFetcher; @@ -66,7 +72,7 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { private ExecutorService executorService; private RejectedExecutionHandler rejectedExecutionHandler; private int numberOfRecords = 10; - private CompletionService completionService; + @Before public void setup() { @@ -80,8 +86,9 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { new LinkedBlockingQueue<>(1), new ThreadFactoryBuilder().setDaemon(true).setNameFormat("getrecords-worker-%d").build(), rejectedExecutionHandler)); - getRecordsRetrivalStrategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, executorService, RETRY_GET_RECORDS_IN_SECONDS, "shardId-0001"); - completionService = spy(getRecordsRetrivalStrategy.completionService); + completionService = spy(new ExecutorCompletionService(executorService)); + when(completionServiceSupplier.get()).thenReturn(completionService); + getRecordsRetrivalStrategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, executorService, RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, "shardId-0001"); result = null; } @@ -97,12 +104,16 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { public void multiRequestTest() { result = mock(GetRecordsResult.class); + ExecutorCompletionService completionService1 = spy(new ExecutorCompletionService(executorService)); + when(completionServiceSupplier.get()).thenReturn(completionService1); GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(numberOfRecords); verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any()); assertEquals(result, getRecordsResult); result = null; + ExecutorCompletionService completionService2 = spy(new ExecutorCompletionService(executorService)); + when(completionServiceSupplier.get()).thenReturn(completionService2); getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); assertNull(getRecordsResult); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java index 9ecea68d..820f4a57 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java @@ -30,7 +30,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -51,6 +53,8 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { @Mock private ExecutorService executorService; @Mock + private Supplier> completionServiceSupplier; + @Mock private CompletionService completionService; @Mock private Future successfulFuture; @@ -59,10 +63,15 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { @Mock private GetRecordsResult expectedResults; + @Before + public void before() { + when(completionServiceSupplier.get()).thenReturn(completionService); + } + @Test public void testSingleSuccessfulRequestFuture() throws Exception { AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, - executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID); when(executorService.isShutdown()).thenReturn(false); when(completionService.submit(any())).thenReturn(successfulFuture); @@ -76,8 +85,6 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { verify(completionService).poll(eq(RETRY_GET_RECORDS_IN_SECONDS), eq(TimeUnit.SECONDS)); verify(successfulFuture).get(); verify(successfulFuture).cancel(eq(true)); - verify(successfulFuture).isCancelled(); - verify(completionService, never()).take(); assertThat(result, equalTo(expectedResults)); } @@ -85,7 +92,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { @Test public void testBlockedAndSuccessfulFuture() throws Exception { AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, - executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID); when(executorService.isShutdown()).thenReturn(false); when(completionService.submit(any())).thenReturn(blockedFuture).thenReturn(successfulFuture); @@ -104,9 +111,6 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { verify(blockedFuture, never()).get(); verify(successfulFuture).cancel(eq(true)); verify(blockedFuture).cancel(eq(true)); - verify(successfulFuture).isCancelled(); - verify(blockedFuture).isCancelled(); - verify(completionService).take(); assertThat(actualResults, equalTo(expectedResults)); } @@ -114,7 +118,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { @Test(expected = IllegalStateException.class) public void testStrategyIsShutdown() throws Exception { AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, - executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID); when(executorService.isShutdown()).thenReturn(true); @@ -124,7 +128,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { @Test public void testPoolOutOfResources() throws Exception { AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, - executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionService, SHARD_ID); + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID); when(executorService.isShutdown()).thenReturn(false); when(completionService.submit(any())).thenReturn(blockedFuture).thenThrow(new RejectedExecutionException("Rejected!")).thenReturn(successfulFuture); @@ -141,9 +145,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { verify(completionService, times(3)).poll(eq(RETRY_GET_RECORDS_IN_SECONDS), eq(TimeUnit.SECONDS)); verify(successfulFuture).cancel(eq(true)); verify(blockedFuture).cancel(eq(true)); - verify(successfulFuture).isCancelled(); - verify(blockedFuture).isCancelled(); - verify(completionService).take(); + assertThat(actualResult, equalTo(expectedResults)); } From c1df703d676b8848136595bd47edec71b8ce62f0 Mon Sep 17 00:00:00 2001 From: "Pfifer, Justin" Date: Fri, 22 Sep 2017 16:23:20 -0700 Subject: [PATCH 10/35] Release 1.8.4 of the Amazon Kinesis Client for Java * Create a new completion service for each request. This ensures that canceled tasks are discarded. This will prevent a cancellation exception causing issues processing records. * PR #227 * Issue #226 --- README.md | 6 ++++++ pom.xml | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index ddaa6194..00ed3caa 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,12 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.4 (September 22, 2017) +* Create a new completion service for each request. + This ensures that canceled tasks are discarded. This will prevent a cancellation exception causing issues processing records. + * [PR #227](https://github.com/awslabs/amazon-kinesis-client/pull/227) + * [Issue #226](https://github.com/awslabs/amazon-kinesis-client/issues/226) + ### Release 1.8.3 (September 22, 2017) * Call shutdown on the retriever when the record processor is being shutdown This fixes a bug that could leak threads if using the [`AsynchronousGetRecordsRetrievalStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/9a82b6bd05b3c9c5f8581af007141fa6d5f0fc4e/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java#L42) is being used. diff --git a/pom.xml b/pom.xml index 5d8bdb53..f0f783a2 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.4-SNAPSHOT + 1.8.4 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 4b20556f373d4cbe02dda34a6b92e8c6d0dfafb7 Mon Sep 17 00:00:00 2001 From: "Pfifer, Justin" Date: Tue, 26 Sep 2017 12:39:53 -0700 Subject: [PATCH 11/35] Only advance the shard iterator when we accept a result to return This changes the retriever strategy to only accept the shard iterator when we have accepted a result to return. This is for the asynchronous retriever where multiple threads may contend for the same iterator slot. This ensures only the one selected for the response will advance the shard iterator. --- pom.xml | 2 +- ...ynchronousGetRecordsRetrievalStrategy.java | 21 ++-- .../lib/worker/DataFetcherResult.java | 37 ++++++++ .../lib/worker/KinesisDataFetcher.java | 59 ++++++++++-- ...ynchronousGetRecordsRetrievalStrategy.java | 2 +- ...cordsRetrievalStrategyIntegrationTest.java | 36 +++---- ...ronousGetRecordsRetrievalStrategyTest.java | 18 ++-- .../lib/worker/KinesisDataFetcherTest.java | 95 +++++++++++++++++++ 8 files changed, 228 insertions(+), 42 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetcherResult.java diff --git a/pom.xml b/pom.xml index f0f783a2..66006321 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.4 + 1.8.5-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java index b592c29b..2e3cbd9e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java @@ -49,7 +49,7 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie private final ExecutorService executorService; private final int retryGetRecordsInSeconds; private final String shardId; - final Supplier> completionServiceSupplier; + final Supplier> completionServiceSupplier; public AsynchronousGetRecordsRetrievalStrategy(@NonNull final KinesisDataFetcher dataFetcher, final int retryGetRecordsInSeconds, final int maxGetRecordsThreadPool, String shardId) { @@ -63,7 +63,7 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie } AsynchronousGetRecordsRetrievalStrategy(KinesisDataFetcher dataFetcher, ExecutorService executorService, - int retryGetRecordsInSeconds, Supplier> completionServiceSupplier, + int retryGetRecordsInSeconds, Supplier> completionServiceSupplier, String shardId) { this.dataFetcher = dataFetcher; this.executorService = executorService; @@ -78,9 +78,9 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie throw new IllegalStateException("Strategy has been shutdown"); } GetRecordsResult result = null; - CompletionService completionService = completionServiceSupplier.get(); - Set> futures = new HashSet<>(); - Callable retrieverCall = createRetrieverCallable(maxRecords); + CompletionService completionService = completionServiceSupplier.get(); + Set> futures = new HashSet<>(); + Callable retrieverCall = createRetrieverCallable(maxRecords); while (true) { try { futures.add(completionService.submit(retrieverCall)); @@ -89,10 +89,15 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie } try { - Future resultFuture = completionService.poll(retryGetRecordsInSeconds, + Future resultFuture = completionService.poll(retryGetRecordsInSeconds, TimeUnit.SECONDS); if (resultFuture != null) { - result = resultFuture.get(); + // + // Fix to ensure that we only let the shard iterator advance when we intend to return the result + // to the caller. This ensures that the shard iterator is consistently advance in step with + // what the caller sees. + // + result = resultFuture.get().accept(); break; } } catch (ExecutionException e) { @@ -106,7 +111,7 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie return result; } - private Callable createRetrieverCallable(int maxRecords) { + private Callable createRetrieverCallable(int maxRecords) { ThreadSafeMetricsDelegatingScope metricsScope = new ThreadSafeMetricsDelegatingScope(MetricsHelper.getMetricsScope()); return () -> { try { diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetcherResult.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetcherResult.java new file mode 100644 index 00000000..a7121ff2 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetcherResult.java @@ -0,0 +1,37 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. Licensed under the Amazon Software License + * (the "License"). You may not use this file except in compliance with the License. A copy of the License is located at + * http://aws.amazon.com/asl/ or in the "license" file accompanying this file. This file is distributed on an "AS IS" + * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific + * language governing permissions and limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; + +/** + * Represents the result from the DataFetcher, and allows the receiver to accept a result + */ +public interface DataFetcherResult { + /** + * The result of the request to Kinesis + * + * @return The result of the request, this can be null if the request failed. + */ + GetRecordsResult getResult(); + + /** + * Accepts the result, and advances the shard iterator. A result from the data fetcher must be accepted before any + * further progress can be made. + * + * @return the result of the request, this can be null if the request failed. + */ + GetRecordsResult accept(); + + /** + * Indicates whether this result is at the end of the shard or not + * + * @return true if the result is at the end of a shard, false otherwise + */ + boolean isShardEnd(); +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java index 2ce3152a..dec0ac5e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java @@ -14,6 +14,7 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import lombok.Data; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -26,6 +27,7 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.MetricsCollectingKin import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import java.util.Date; +import java.util.function.Consumer; /** * Used to get data from Amazon Kinesis. Tracks iterator state internally. @@ -57,30 +59,69 @@ class KinesisDataFetcher { * @param maxRecords Max records to fetch * @return list of records of up to maxRecords size */ - public GetRecordsResult getRecords(int maxRecords) { + public DataFetcherResult getRecords(int maxRecords) { if (!isInitialized) { throw new IllegalArgumentException("KinesisDataFetcher.getRecords called before initialization."); } - GetRecordsResult response = null; + DataFetcherResult response; if (nextIterator != null) { try { - response = kinesisProxy.get(nextIterator, maxRecords); - nextIterator = response.getNextShardIterator(); + response = new AdvancingResult(kinesisProxy.get(nextIterator, maxRecords)); } catch (ResourceNotFoundException e) { LOG.info("Caught ResourceNotFoundException when fetching records for shard " + shardId); - nextIterator = null; - } - if (nextIterator == null) { - isShardEndReached = true; + response = TERMINAL_RESULT; } } else { - isShardEndReached = true; + response = TERMINAL_RESULT; } return response; } + final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() { + @Override + public GetRecordsResult getResult() { + return null; + } + + @Override + public GetRecordsResult accept() { + isShardEndReached = true; + return getResult(); + } + + @Override + public boolean isShardEnd() { + return isShardEndReached; + } + }; + + @Data + private class AdvancingResult implements DataFetcherResult { + + final GetRecordsResult result; + + @Override + public GetRecordsResult getResult() { + return result; + } + + @Override + public GetRecordsResult accept() { + nextIterator = result.getNextShardIterator(); + if (nextIterator == null) { + isShardEndReached = true; + } + return getResult(); + } + + @Override + public boolean isShardEnd() { + return isShardEndReached; + } + } + /** * Initializes this KinesisDataFetcher's iterator based on the checkpointed sequence number. * @param initialCheckpoint Current checkpoint sequence number for this shard. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java index 3c8925b0..c862c348 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java @@ -28,7 +28,7 @@ public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetriev @Override public GetRecordsResult getRecords(final int maxRecords) { - return dataFetcher.getRecords(maxRecords); + return dataFetcher.getRecords(maxRecords).accept(); } @Override diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java index b3659605..8e89204e 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java @@ -36,7 +36,10 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; -import static org.junit.Assert.assertEquals; + +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertNull; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; @@ -58,17 +61,19 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { @Mock private IKinesisProxy mockKinesisProxy; - @Mock private ShardInfo mockShardInfo; @Mock - private Supplier> completionServiceSupplier; + private Supplier> completionServiceSupplier; + @Mock + private DataFetcherResult result; + @Mock + private GetRecordsResult recordsResult; - private CompletionService completionService; + private CompletionService completionService; private AsynchronousGetRecordsRetrievalStrategy getRecordsRetrivalStrategy; private KinesisDataFetcher dataFetcher; - private GetRecordsResult result; private ExecutorService executorService; private RejectedExecutionHandler rejectedExecutionHandler; private int numberOfRecords = 10; @@ -86,14 +91,15 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { new LinkedBlockingQueue<>(1), new ThreadFactoryBuilder().setDaemon(true).setNameFormat("getrecords-worker-%d").build(), rejectedExecutionHandler)); - completionService = spy(new ExecutorCompletionService(executorService)); + completionService = spy(new ExecutorCompletionService(executorService)); when(completionServiceSupplier.get()).thenReturn(completionService); getRecordsRetrivalStrategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, executorService, RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, "shardId-0001"); - result = null; + when(result.accept()).thenReturn(recordsResult); } @Test public void oneRequestMultithreadTest() { + when(result.accept()).thenReturn(null); GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(eq(numberOfRecords)); verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any()); @@ -102,27 +108,25 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { @Test public void multiRequestTest() { - result = mock(GetRecordsResult.class); - - ExecutorCompletionService completionService1 = spy(new ExecutorCompletionService(executorService)); + ExecutorCompletionService completionService1 = spy(new ExecutorCompletionService(executorService)); when(completionServiceSupplier.get()).thenReturn(completionService1); GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(numberOfRecords); verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any()); - assertEquals(result, getRecordsResult); + assertThat(getRecordsResult, equalTo(recordsResult)); - result = null; - ExecutorCompletionService completionService2 = spy(new ExecutorCompletionService(executorService)); + when(result.accept()).thenReturn(null); + ExecutorCompletionService completionService2 = spy(new ExecutorCompletionService(executorService)); when(completionServiceSupplier.get()).thenReturn(completionService2); getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords); - assertNull(getRecordsResult); + assertThat(getRecordsResult, nullValue(GetRecordsResult.class)); } @Test @Ignore public void testInterrupted() throws InterruptedException, ExecutionException { - Future mockFuture = mock(Future.class); + Future mockFuture = mock(Future.class); when(completionService.submit(any())).thenReturn(mockFuture); when(completionService.poll()).thenReturn(mockFuture); doThrow(InterruptedException.class).when(mockFuture).get(); @@ -154,7 +158,7 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { } @Override - public GetRecordsResult getRecords(final int maxRecords) { + public DataFetcherResult getRecords(final int maxRecords) { try { Thread.sleep(SLEEP_GET_RECORDS_IN_SECONDS * 1000); } catch (InterruptedException e) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java index 820f4a57..aa9e9a24 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java @@ -53,19 +53,23 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { @Mock private ExecutorService executorService; @Mock - private Supplier> completionServiceSupplier; + private Supplier> completionServiceSupplier; @Mock - private CompletionService completionService; + private CompletionService completionService; @Mock - private Future successfulFuture; + private Future successfulFuture; @Mock - private Future blockedFuture; + private Future blockedFuture; + @Mock + private DataFetcherResult dataFetcherResult; @Mock private GetRecordsResult expectedResults; @Before public void before() { when(completionServiceSupplier.get()).thenReturn(completionService); + when(dataFetcherResult.getResult()).thenReturn(expectedResults); + when(dataFetcherResult.accept()).thenReturn(expectedResults); } @Test @@ -76,7 +80,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { when(executorService.isShutdown()).thenReturn(false); when(completionService.submit(any())).thenReturn(successfulFuture); when(completionService.poll(anyLong(), any())).thenReturn(successfulFuture); - when(successfulFuture.get()).thenReturn(expectedResults); + when(successfulFuture.get()).thenReturn(dataFetcherResult); GetRecordsResult result = strategy.getRecords(10); @@ -97,7 +101,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { when(executorService.isShutdown()).thenReturn(false); when(completionService.submit(any())).thenReturn(blockedFuture).thenReturn(successfulFuture); when(completionService.poll(anyLong(), any())).thenReturn(null).thenReturn(successfulFuture); - when(successfulFuture.get()).thenReturn(expectedResults); + when(successfulFuture.get()).thenReturn(dataFetcherResult); when(successfulFuture.cancel(anyBoolean())).thenReturn(false); when(blockedFuture.cancel(anyBoolean())).thenReturn(true); when(successfulFuture.isCancelled()).thenReturn(false); @@ -133,7 +137,7 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { when(executorService.isShutdown()).thenReturn(false); when(completionService.submit(any())).thenReturn(blockedFuture).thenThrow(new RejectedExecutionException("Rejected!")).thenReturn(successfulFuture); when(completionService.poll(anyLong(), any())).thenReturn(null).thenReturn(null).thenReturn(successfulFuture); - when(successfulFuture.get()).thenReturn(expectedResults); + when(successfulFuture.get()).thenReturn(dataFetcherResult); when(successfulFuture.cancel(anyBoolean())).thenReturn(false); when(blockedFuture.cancel(anyBoolean())).thenReturn(true); when(successfulFuture.isCancelled()).thenReturn(false); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java index 2b89c3c8..3cc8cb5a 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java @@ -14,9 +14,20 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.ArrayList; @@ -39,12 +50,19 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; /** * Unit tests for KinesisDataFetcher. */ +@RunWith(MockitoJUnitRunner.class) public class KinesisDataFetcherTest { + @Mock + private KinesisProxy kinesisProxy; + private static final int MAX_RECORDS = 1; private static final String SHARD_ID = "shardId-1"; private static final String AT_SEQUENCE_NUMBER = ShardIteratorType.AT_SEQUENCE_NUMBER.toString(); @@ -55,6 +73,7 @@ public class KinesisDataFetcherTest { InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); private static final InitialPositionInStreamExtended INITIAL_POSITION_AT_TIMESTAMP = InitialPositionInStreamExtended.newInitialPositionAtTimestamp(new Date(1000)); + ; /** * @throws java.lang.Exception @@ -190,6 +209,82 @@ public class KinesisDataFetcherTest { Assert.assertTrue("Shard should reach the end", dataFetcher.isShardEndReached()); } + @Test + public void testFetcherDoesNotAdvanceWithoutAccept() { + final String INITIAL_ITERATOR = "InitialIterator"; + final String NEXT_ITERATOR_ONE = "NextIteratorOne"; + final String NEXT_ITERATOR_TWO = "NextIteratorTwo"; + when(kinesisProxy.getIterator(anyString(), anyString())).thenReturn(INITIAL_ITERATOR); + GetRecordsResult iteratorOneResults = mock(GetRecordsResult.class); + when(iteratorOneResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_ONE); + when(kinesisProxy.get(eq(INITIAL_ITERATOR), anyInt())).thenReturn(iteratorOneResults); + + GetRecordsResult iteratorTwoResults = mock(GetRecordsResult.class); + when(kinesisProxy.get(eq(NEXT_ITERATOR_ONE), anyInt())).thenReturn(iteratorTwoResults); + when(iteratorTwoResults.getNextShardIterator()).thenReturn(NEXT_ITERATOR_TWO); + + GetRecordsResult finalResult = mock(GetRecordsResult.class); + when(kinesisProxy.get(eq(NEXT_ITERATOR_TWO), anyInt())).thenReturn(finalResult); + when(finalResult.getNextShardIterator()).thenReturn(null); + + + KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); + dataFetcher.initialize("TRIM_HORIZON", InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON)); + + assertNoAdvance(dataFetcher, iteratorOneResults, INITIAL_ITERATOR); + assertAdvanced(dataFetcher, iteratorOneResults, INITIAL_ITERATOR, NEXT_ITERATOR_ONE); + + assertNoAdvance(dataFetcher, iteratorTwoResults, NEXT_ITERATOR_ONE); + assertAdvanced(dataFetcher, iteratorTwoResults, NEXT_ITERATOR_ONE, NEXT_ITERATOR_TWO); + + assertNoAdvance(dataFetcher, finalResult, NEXT_ITERATOR_TWO); + assertAdvanced(dataFetcher, finalResult, NEXT_ITERATOR_TWO, null); + + verify(kinesisProxy, times(2)).get(eq(INITIAL_ITERATOR), anyInt()); + verify(kinesisProxy, times(2)).get(eq(NEXT_ITERATOR_ONE), anyInt()); + verify(kinesisProxy, times(2)).get(eq(NEXT_ITERATOR_TWO), anyInt()); + + reset(kinesisProxy); + + DataFetcherResult terminal = dataFetcher.getRecords(100); + assertThat(terminal.isShardEnd(), equalTo(true)); + assertThat(terminal.getResult(), nullValue()); + assertThat(terminal, equalTo(dataFetcher.TERMINAL_RESULT)); + + verify(kinesisProxy, never()).get(anyString(), anyInt()); + } + + + private DataFetcherResult assertAdvanced(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, String previousValue, String nextValue) { + DataFetcherResult acceptResult = dataFetcher.getRecords(100); + assertThat(acceptResult.getResult(), equalTo(expectedResult)); + + assertThat(dataFetcher.getNextIterator(), equalTo(previousValue)); + assertThat(dataFetcher.isShardEndReached(), equalTo(false)); + + assertThat(acceptResult.accept(), equalTo(expectedResult)); + assertThat(dataFetcher.getNextIterator(), equalTo(nextValue)); + if (nextValue == null) { + assertThat(dataFetcher.isShardEndReached(), equalTo(true)); + } + + verify(kinesisProxy, times(2)).get(eq(previousValue), anyInt()); + + return acceptResult; + } + + private DataFetcherResult assertNoAdvance(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, String previousValue) { + assertThat(dataFetcher.getNextIterator(), equalTo(previousValue)); + DataFetcherResult noAcceptResult = dataFetcher.getRecords(100); + assertThat(noAcceptResult.getResult(), equalTo(expectedResult)); + + assertThat(dataFetcher.getNextIterator(), equalTo(previousValue)); + + verify(kinesisProxy).get(eq(previousValue), anyInt()); + + return noAcceptResult; + } + private void testInitializeAndFetch(String iteratorType, String seqNo, InitialPositionInStreamExtended initialPositionInStream) throws Exception { From 9720b1b24995605248232df1e1ae7fb9fed4fc98 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Tue, 26 Sep 2017 14:31:59 -0700 Subject: [PATCH 12/35] Release 1.8.5 of the Amazon Kinesis Client for Java (#232) * Release 1.8.5 of the Amazon Kinesis Client for Java Release 1.8.5 (September 26, 2017) * Only advance the shard iterator for the accepted response. This fixes a race condition in the `KinesisDataFetcher` when it's being used to make asynchronous requests. The shard iterator is now only advanced when the retriever calls `DataFetcherResult#accept()`. * PR #230 * Issue #231 --- META-INF/MANIFEST.MF | 2 +- README.md | 6 ++++++ pom.xml | 2 +- .../lib/worker/KinesisClientLibConfiguration.java | 2 +- 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index a7ac9a5a..146a18fe 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.4 +Bundle-Version: 1.8.5 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.7 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/README.md b/README.md index 00ed3caa..8191254a 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,12 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.5 (September 26, 2017) +* Only advance the shard iterator for the accepted response. + This fixes a race condition in the `KinesisDataFetcher` when it's being used to make asynchronous requests. The shard iterator is now only advanced when the retriever calls `DataFetcherResult#accept()`. + * [PR #230](https://github.com/awslabs/amazon-kinesis-client/pull/230) + * [Issue #231](https://github.com/awslabs/amazon-kinesis-client/issues/231) + ### Release 1.8.4 (September 22, 2017) * Create a new completion service for each request. This ensures that canceled tasks are discarded. This will prevent a cancellation exception causing issues processing records. diff --git a/pom.xml b/pom.xml index 66006321..895c6542 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.5-SNAPSHOT + 1.8.5 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 4620bfe4..7d6dac5a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.4"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.5"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From ee3a6c24e68f0221b0da8d7db0fea5cfebc11362 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Tue, 17 Oct 2017 13:05:43 -0700 Subject: [PATCH 13/35] Support Prefetching of Records This adds that ability for the KCL to fetch records while the record processor is busy. This can help smooth out delays in record process, or retrieving data from Kinesis. Enabling this does require extra threads for background retrieval. Settings * dataFetchingStrategy: Which strategy to use to retrieve records. This can be either DEFAULT or PREFETCH_CACHED * maxCacheByteSize: Retrieval will be paused when the total number of bytes in the cache exceeds this value * maxRecordsCount: Retrieval will be paused when the total number of records in the cache exceeds this value * maxPendingProcessRecordsInput: Retrieval will be paused when the total number of fulfilled requests in the cache exceeds this value --- pom.xml | 2 +- .../lib/worker/BlockingGetRecordsCache.java | 90 ++++++ .../lib/worker/ConsumerStates.java | 42 ++- .../lib/worker/DataFetchingStrategy.java | 8 + .../lib/worker/GetRecordsCache.java | 43 +++ .../lib/worker/GetRecordsRetriever.java | 12 + .../lib/worker/InitializeTask.java | 34 +- .../worker/KinesisClientLibConfiguration.java | 267 +++++++++++++--- .../lib/worker/KinesisDataFetcher.java | 50 ++- .../lib/worker/PrefetchGetRecordsCache.java | 223 ++++++++++++++ .../clientlibrary/lib/worker/ProcessTask.java | 63 ++-- .../lib/worker/RecordsFetcherFactory.java | 74 +++++ .../lib/worker/ShardConsumer.java | 132 ++++++-- .../lib/worker/ShutdownTask.java | 8 +- .../worker/SimpleRecordsFetcherFactory.java | 77 +++++ .../clientlibrary/lib/worker/Worker.java | 47 ++- .../types/ProcessRecordsInput.java | 27 +- ...cordsRetrievalStrategyIntegrationTest.java | 47 ++- .../worker/BlockingGetRecordsCacheTest.java | 84 +++++ .../lib/worker/ConsumerStatesTest.java | 43 ++- .../KinesisClientLibConfigurationTest.java | 55 ++-- .../lib/worker/KinesisDataFetcherTest.java | 51 ++- ...refetchGetRecordsCacheIntegrationTest.java | 199 ++++++++++++ .../worker/PrefetchGetRecordsCacheTest.java | 215 +++++++++++++ .../lib/worker/ProcessTaskTest.java | 34 +- .../lib/worker/RecordsFetcherFactoryTest.java | 44 +++ .../lib/worker/ShardConsumerTest.java | 112 ++++++- .../lib/worker/ShutdownTaskTest.java | 12 +- .../clientlibrary/lib/worker/WorkerTest.java | 291 +++++++++++++----- 29 files changed, 2024 insertions(+), 362 deletions(-) create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetchingStrategy.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsCache.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetriever.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java create mode 100644 src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java create mode 100644 src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java diff --git a/pom.xml b/pom.xml index 895c6542..0e40b5ab 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.5 + 1.8.6-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java new file mode 100644 index 00000000..d9fc011e --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java @@ -0,0 +1,90 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import java.time.Duration; +import java.time.Instant; + +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.model.GetRecordsResult; + +import lombok.extern.apachecommons.CommonsLog; + +/** + * This is the BlockingGetRecordsCache class. This class blocks any calls to the getRecords on the + * GetRecordsRetrievalStrategy class. + */ +@CommonsLog +public class BlockingGetRecordsCache implements GetRecordsCache { + private final int maxRecordsPerCall; + private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private final long idleMillisBetweenCalls; + private Instant lastSuccessfulCall; + + public BlockingGetRecordsCache(final int maxRecordsPerCall, + final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, + final long idleMillisBetweenCalls) { + this.maxRecordsPerCall = maxRecordsPerCall; + this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; + this.idleMillisBetweenCalls = idleMillisBetweenCalls; + } + + @Override + public void start() { + // + // Nothing to do here + // + } + + @Override + public ProcessRecordsInput getNextResult() { + sleepBeforeNextCall(); + GetRecordsResult getRecordsResult = getRecordsRetrievalStrategy.getRecords(maxRecordsPerCall); + lastSuccessfulCall = Instant.now(); + ProcessRecordsInput processRecordsInput = new ProcessRecordsInput() + .withRecords(getRecordsResult.getRecords()) + .withMillisBehindLatest(getRecordsResult.getMillisBehindLatest()); + return processRecordsInput; + } + + private void sleepBeforeNextCall() { + if (!Thread.interrupted()) { + if (lastSuccessfulCall == null) { + return; + } + long timeSinceLastCall = Duration.between(lastSuccessfulCall, Instant.now()).abs().toMillis(); + if (timeSinceLastCall < idleMillisBetweenCalls) { + try { + Thread.sleep(idleMillisBetweenCalls - timeSinceLastCall); + } catch (InterruptedException e) { + log.info("Thread was interrupted, indicating that shutdown was called."); + } + } + } else { + log.info("Thread has been interrupted, indicating that it is in the shutdown phase."); + } + } + + @Override + public GetRecordsRetrievalStrategy getGetRecordsRetrievalStrategy() { + return getRecordsRetrievalStrategy; + } + + @Override + public void shutdown() { + getRecordsRetrievalStrategy.shutdown(); + } +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index d3ccb911..9121df4b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -14,8 +14,6 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; -import java.util.Optional; - /** * Top level container for all the possible states a {@link ShardConsumer} can be in. The logic for creation of tasks, * and state transitions is contained within the {@link ConsumerState} objects. @@ -253,9 +251,14 @@ class ConsumerStates { @Override public ITask createTask(ShardConsumer consumer) { - return new InitializeTask(consumer.getShardInfo(), consumer.getRecordProcessor(), consumer.getCheckpoint(), - consumer.getRecordProcessorCheckpointer(), consumer.getDataFetcher(), - consumer.getTaskBackoffTimeMillis(), consumer.getStreamConfig()); + return new InitializeTask(consumer.getShardInfo(), + consumer.getRecordProcessor(), + consumer.getCheckpoint(), + consumer.getRecordProcessorCheckpointer(), + consumer.getDataFetcher(), + consumer.getTaskBackoffTimeMillis(), + consumer.getStreamConfig(), + consumer.getGetRecordsCache()); } @Override @@ -309,10 +312,14 @@ class ConsumerStates { @Override public ITask createTask(ShardConsumer consumer) { - return new ProcessTask(consumer.getShardInfo(), consumer.getStreamConfig(), consumer.getRecordProcessor(), - consumer.getRecordProcessorCheckpointer(), consumer.getDataFetcher(), - consumer.getTaskBackoffTimeMillis(), consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist(), - consumer.getGetRecordsRetrievalStrategy()); + return new ProcessTask(consumer.getShardInfo(), + consumer.getStreamConfig(), + consumer.getRecordProcessor(), + consumer.getRecordProcessorCheckpointer(), + consumer.getDataFetcher(), + consumer.getTaskBackoffTimeMillis(), + consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist(), + consumer.getGetRecordsCache()); } @Override @@ -371,8 +378,10 @@ class ConsumerStates { @Override public ITask createTask(ShardConsumer consumer) { - return new ShutdownNotificationTask(consumer.getRecordProcessor(), consumer.getRecordProcessorCheckpointer(), - consumer.getShutdownNotification(), consumer.getShardInfo()); + return new ShutdownNotificationTask(consumer.getRecordProcessor(), + consumer.getRecordProcessorCheckpointer(), + consumer.getShutdownNotification(), + consumer.getShardInfo()); } @Override @@ -511,13 +520,16 @@ class ConsumerStates { @Override public ITask createTask(ShardConsumer consumer) { - return new ShutdownTask(consumer.getShardInfo(), consumer.getRecordProcessor(), - consumer.getRecordProcessorCheckpointer(), consumer.getShutdownReason(), + return new ShutdownTask(consumer.getShardInfo(), + consumer.getRecordProcessor(), + consumer.getRecordProcessorCheckpointer(), + consumer.getShutdownReason(), consumer.getStreamConfig().getStreamProxy(), consumer.getStreamConfig().getInitialPositionInStream(), - consumer.isCleanupLeasesOfCompletedShards(), consumer.getLeaseManager(), + consumer.isCleanupLeasesOfCompletedShards(), + consumer.getLeaseManager(), consumer.getTaskBackoffTimeMillis(), - consumer.getGetRecordsRetrievalStrategy()); + consumer.getGetRecordsCache()); } @Override diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetchingStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetchingStrategy.java new file mode 100644 index 00000000..05c2ab3f --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/DataFetchingStrategy.java @@ -0,0 +1,8 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +/** + * + */ +public enum DataFetchingStrategy { + DEFAULT, PREFETCH_CACHED; +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsCache.java new file mode 100644 index 00000000..dba24f8d --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsCache.java @@ -0,0 +1,43 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; + +/** + * This class is used as a cache for Prefetching data from Kinesis. + */ +public interface GetRecordsCache { + /** + * This method calls the start behavior on the cache, if available. + */ + void start(); + + /** + * This method returns the next set of records from the Cache if present, or blocks the request till it gets the + * next set of records back from Kinesis. + * + * @return The next set of records. + */ + ProcessRecordsInput getNextResult(); + + GetRecordsRetrievalStrategy getGetRecordsRetrievalStrategy(); + + /** + * This method calls the shutdown behavior on the cache, if available. + */ + void shutdown(); +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetriever.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetriever.java new file mode 100644 index 00000000..d5b4a782 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetriever.java @@ -0,0 +1,12 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; + +import java.util.concurrent.Callable; + +/** + * This class uses the GetRecordsRetrievalStrategy class to retrieve the next set of records and update the cache. + */ +public interface GetRecordsRetriever { + GetRecordsResult getNextRecords(int maxRecords); +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java index e3d9f607..5e847a89 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/InitializeTask.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; @@ -43,17 +43,19 @@ class InitializeTask implements ITask { // Back off for this interval if we encounter a problem (exception) private final long backoffTimeMillis; private final StreamConfig streamConfig; + private final GetRecordsCache getRecordsCache; /** * Constructor. */ InitializeTask(ShardInfo shardInfo, - IRecordProcessor recordProcessor, - ICheckpoint checkpoint, - RecordProcessorCheckpointer recordProcessorCheckpointer, - KinesisDataFetcher dataFetcher, - long backoffTimeMillis, - StreamConfig streamConfig) { + IRecordProcessor recordProcessor, + ICheckpoint checkpoint, + RecordProcessorCheckpointer recordProcessorCheckpointer, + KinesisDataFetcher dataFetcher, + long backoffTimeMillis, + StreamConfig streamConfig, + GetRecordsCache getRecordsCache) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.checkpoint = checkpoint; @@ -61,6 +63,7 @@ class InitializeTask implements ITask { this.dataFetcher = dataFetcher; this.backoffTimeMillis = backoffTimeMillis; this.streamConfig = streamConfig; + this.getRecordsCache = getRecordsCache; } /* @@ -80,6 +83,7 @@ class InitializeTask implements ITask { ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.getCheckpoint(); dataFetcher.initialize(initialCheckpoint.getSequenceNumber(), streamConfig.getInitialPositionInStream()); + getRecordsCache.start(); recordProcessorCheckpointer.setLargestPermittedCheckpointValue(initialCheckpoint); recordProcessorCheckpointer.setInitialCheckpointValue(initialCheckpoint); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 7d6dac5a..b0ac6bfd 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; @@ -232,6 +232,9 @@ public class KinesisClientLibConfiguration { @Getter private int maxLeaseRenewalThreads = DEFAULT_MAX_LEASE_RENEWAL_THREADS; + @Getter + private RecordsFetcherFactory recordsFetcherFactory; + /** * Constructor. * @@ -267,14 +270,30 @@ public class KinesisClientLibConfiguration { AWSCredentialsProvider dynamoDBCredentialsProvider, AWSCredentialsProvider cloudWatchCredentialsProvider, String workerId) { - this(applicationName, streamName, null, null, DEFAULT_INITIAL_POSITION_IN_STREAM, kinesisCredentialsProvider, - dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, DEFAULT_FAILOVER_TIME_MILLIS, workerId, - DEFAULT_MAX_RECORDS, DEFAULT_IDLETIME_BETWEEN_READS_MILLIS, - DEFAULT_DONT_CALL_PROCESS_RECORDS_FOR_EMPTY_RECORD_LIST, DEFAULT_PARENT_SHARD_POLL_INTERVAL_MILLIS, - DEFAULT_SHARD_SYNC_INTERVAL_MILLIS, DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION, - new ClientConfiguration(), new ClientConfiguration(), new ClientConfiguration(), - DEFAULT_TASK_BACKOFF_TIME_MILLIS, DEFAULT_METRICS_BUFFER_TIME_MILLIS, DEFAULT_METRICS_MAX_QUEUE_SIZE, - DEFAULT_VALIDATE_SEQUENCE_NUMBER_BEFORE_CHECKPOINTING, null, + this(applicationName, + streamName, + null, + null, + DEFAULT_INITIAL_POSITION_IN_STREAM, + kinesisCredentialsProvider, + dynamoDBCredentialsProvider, + cloudWatchCredentialsProvider, + DEFAULT_FAILOVER_TIME_MILLIS, + workerId, + DEFAULT_MAX_RECORDS, + DEFAULT_IDLETIME_BETWEEN_READS_MILLIS, + DEFAULT_DONT_CALL_PROCESS_RECORDS_FOR_EMPTY_RECORD_LIST, + DEFAULT_PARENT_SHARD_POLL_INTERVAL_MILLIS, + DEFAULT_SHARD_SYNC_INTERVAL_MILLIS, + DEFAULT_CLEANUP_LEASES_UPON_SHARDS_COMPLETION, + new ClientConfiguration(), + new ClientConfiguration(), + new ClientConfiguration(), + DEFAULT_TASK_BACKOFF_TIME_MILLIS, + DEFAULT_METRICS_BUFFER_TIME_MILLIS, + DEFAULT_METRICS_MAX_QUEUE_SIZE, + DEFAULT_VALIDATE_SEQUENCE_NUMBER_BEFORE_CHECKPOINTING, + null, DEFAULT_SHUTDOWN_GRACE_MILLIS); } @@ -315,29 +334,29 @@ public class KinesisClientLibConfiguration { // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES public KinesisClientLibConfiguration(String applicationName, - String streamName, - String kinesisEndpoint, - InitialPositionInStream initialPositionInStream, - AWSCredentialsProvider kinesisCredentialsProvider, - AWSCredentialsProvider dynamoDBCredentialsProvider, - AWSCredentialsProvider cloudWatchCredentialsProvider, - long failoverTimeMillis, - String workerId, - int maxRecords, - long idleTimeBetweenReadsInMillis, - boolean callProcessRecordsEvenForEmptyRecordList, - long parentShardPollIntervalMillis, - long shardSyncIntervalMillis, - boolean cleanupTerminatedShardsBeforeExpiry, - ClientConfiguration kinesisClientConfig, - ClientConfiguration dynamoDBClientConfig, - ClientConfiguration cloudWatchClientConfig, - long taskBackoffTimeMillis, - long metricsBufferTimeMillis, - int metricsMaxQueueSize, - boolean validateSequenceNumberBeforeCheckpointing, - String regionName, - long shutdownGraceMillis) { + String streamName, + String kinesisEndpoint, + InitialPositionInStream initialPositionInStream, + AWSCredentialsProvider kinesisCredentialsProvider, + AWSCredentialsProvider dynamoDBCredentialsProvider, + AWSCredentialsProvider cloudWatchCredentialsProvider, + long failoverTimeMillis, + String workerId, + int maxRecords, + long idleTimeBetweenReadsInMillis, + boolean callProcessRecordsEvenForEmptyRecordList, + long parentShardPollIntervalMillis, + long shardSyncIntervalMillis, + boolean cleanupTerminatedShardsBeforeExpiry, + ClientConfiguration kinesisClientConfig, + ClientConfiguration dynamoDBClientConfig, + ClientConfiguration cloudWatchClientConfig, + long taskBackoffTimeMillis, + long metricsBufferTimeMillis, + int metricsMaxQueueSize, + boolean validateSequenceNumberBeforeCheckpointing, + String regionName, + long shutdownGraceMillis) { this(applicationName, streamName, kinesisEndpoint, null, initialPositionInStream, kinesisCredentialsProvider, dynamoDBCredentialsProvider, cloudWatchCredentialsProvider, failoverTimeMillis, workerId, maxRecords, idleTimeBetweenReadsInMillis, @@ -345,7 +364,117 @@ public class KinesisClientLibConfiguration { shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry, kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig, taskBackoffTimeMillis, metricsBufferTimeMillis, metricsMaxQueueSize, - validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis); + validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis); + } + + /** + * @param applicationName Name of the Kinesis application + * By default the application name is included in the user agent string used to make AWS requests. This + * can assist with troubleshooting (e.g. distinguish requests made by separate applications). + * @param streamName Name of the Kinesis stream + * @param kinesisEndpoint Kinesis endpoint + * @param dynamoDBEndpoint DynamoDB endpoint + * @param initialPositionInStream One of LATEST or TRIM_HORIZON. The KinesisClientLibrary will start fetching + * records from that location in the stream when an application starts up for the first time and there + * are no checkpoints. If there are checkpoints, then we start from the checkpoint position. + * @param kinesisCredentialsProvider Provides credentials used to access Kinesis + * @param dynamoDBCredentialsProvider Provides credentials used to access DynamoDB + * @param cloudWatchCredentialsProvider Provides credentials used to access CloudWatch + * @param failoverTimeMillis Lease duration (leases not renewed within this period will be claimed by others) + * @param workerId Used to distinguish different workers/processes of a Kinesis application + * @param maxRecords Max records to read per Kinesis getRecords() call + * @param idleTimeBetweenReadsInMillis Idle time between calls to fetch data from Kinesis + * @param callProcessRecordsEvenForEmptyRecordList Call the IRecordProcessor::processRecords() API even if + * GetRecords returned an empty record list. + * @param parentShardPollIntervalMillis Wait for this long between polls to check if parent shards are done + * @param shardSyncIntervalMillis Time between tasks to sync leases and Kinesis shards + * @param cleanupTerminatedShardsBeforeExpiry Clean up shards we've finished processing (don't wait for expiration + * in Kinesis) + * @param kinesisClientConfig Client Configuration used by Kinesis client + * @param dynamoDBClientConfig Client Configuration used by DynamoDB client + * @param cloudWatchClientConfig Client Configuration used by CloudWatch client + * @param taskBackoffTimeMillis Backoff period when tasks encounter an exception + * @param metricsBufferTimeMillis Metrics are buffered for at most this long before publishing to CloudWatch + * @param metricsMaxQueueSize Max number of metrics to buffer before publishing to CloudWatch + * @param validateSequenceNumberBeforeCheckpointing whether KCL should validate client provided sequence numbers + * with a call to Amazon Kinesis before checkpointing for calls to + * {@link RecordProcessorCheckpointer#checkpoint(String)} + * @param regionName The region name for the service + */ + // CHECKSTYLE:IGNORE HiddenFieldCheck FOR NEXT 26 LINES + // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 26 LINES + public KinesisClientLibConfiguration(String applicationName, + String streamName, + String kinesisEndpoint, + String dynamoDBEndpoint, + InitialPositionInStream initialPositionInStream, + AWSCredentialsProvider kinesisCredentialsProvider, + AWSCredentialsProvider dynamoDBCredentialsProvider, + AWSCredentialsProvider cloudWatchCredentialsProvider, + long failoverTimeMillis, + String workerId, + int maxRecords, + long idleTimeBetweenReadsInMillis, + boolean callProcessRecordsEvenForEmptyRecordList, + long parentShardPollIntervalMillis, + long shardSyncIntervalMillis, + boolean cleanupTerminatedShardsBeforeExpiry, + ClientConfiguration kinesisClientConfig, + ClientConfiguration dynamoDBClientConfig, + ClientConfiguration cloudWatchClientConfig, + long taskBackoffTimeMillis, + long metricsBufferTimeMillis, + int metricsMaxQueueSize, + boolean validateSequenceNumberBeforeCheckpointing, + String regionName, + long shutdownGraceMillis) { + // Check following values are greater than zero + checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis); + checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis); + checkIsValuePositive("ParentShardPollIntervalMillis", parentShardPollIntervalMillis); + checkIsValuePositive("ShardSyncIntervalMillis", shardSyncIntervalMillis); + checkIsValuePositive("MaxRecords", (long) maxRecords); + checkIsValuePositive("TaskBackoffTimeMillis", taskBackoffTimeMillis); + checkIsValuePositive("MetricsBufferTimeMills", metricsBufferTimeMillis); + checkIsValuePositive("MetricsMaxQueueSize", (long) metricsMaxQueueSize); + checkIsValuePositive("ShutdownGraceMillis", shutdownGraceMillis); + checkIsRegionNameValid(regionName); + this.applicationName = applicationName; + this.tableName = applicationName; + this.streamName = streamName; + this.kinesisEndpoint = kinesisEndpoint; + this.dynamoDBEndpoint = dynamoDBEndpoint; + this.initialPositionInStream = initialPositionInStream; + this.kinesisCredentialsProvider = kinesisCredentialsProvider; + this.dynamoDBCredentialsProvider = dynamoDBCredentialsProvider; + this.cloudWatchCredentialsProvider = cloudWatchCredentialsProvider; + this.failoverTimeMillis = failoverTimeMillis; + this.maxRecords = maxRecords; + this.idleTimeBetweenReadsInMillis = idleTimeBetweenReadsInMillis; + this.callProcessRecordsEvenForEmptyRecordList = callProcessRecordsEvenForEmptyRecordList; + this.parentShardPollIntervalMillis = parentShardPollIntervalMillis; + this.shardSyncIntervalMillis = shardSyncIntervalMillis; + this.cleanupLeasesUponShardCompletion = cleanupTerminatedShardsBeforeExpiry; + this.workerIdentifier = workerId; + this.kinesisClientConfig = checkAndAppendKinesisClientLibUserAgent(kinesisClientConfig); + this.dynamoDBClientConfig = checkAndAppendKinesisClientLibUserAgent(dynamoDBClientConfig); + this.cloudWatchClientConfig = checkAndAppendKinesisClientLibUserAgent(cloudWatchClientConfig); + this.taskBackoffTimeMillis = taskBackoffTimeMillis; + this.metricsBufferTimeMillis = metricsBufferTimeMillis; + this.metricsMaxQueueSize = metricsMaxQueueSize; + this.metricsLevel = DEFAULT_METRICS_LEVEL; + this.metricsEnabledDimensions = DEFAULT_METRICS_ENABLED_DIMENSIONS; + this.validateSequenceNumberBeforeCheckpointing = validateSequenceNumberBeforeCheckpointing; + this.regionName = regionName; + this.maxLeasesForWorker = DEFAULT_MAX_LEASES_FOR_WORKER; + this.maxLeasesToStealAtOneTime = DEFAULT_MAX_LEASES_TO_STEAL_AT_ONE_TIME; + this.initialLeaseTableReadCapacity = DEFAULT_INITIAL_LEASE_TABLE_READ_CAPACITY; + this.initialLeaseTableWriteCapacity = DEFAULT_INITIAL_LEASE_TABLE_WRITE_CAPACITY; + this.initialPositionInStreamExtended = + InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); + this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; + this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; + this.recordsFetcherFactory = new SimpleRecordsFetcherFactory(this.maxRecords); } /** @@ -408,7 +537,7 @@ public class KinesisClientLibConfiguration { int metricsMaxQueueSize, boolean validateSequenceNumberBeforeCheckpointing, String regionName, - long shutdownGraceMillis) { + RecordsFetcherFactory recordsFetcherFactory) { // Check following values are greater than zero checkIsValuePositive("FailoverTimeMillis", failoverTimeMillis); checkIsValuePositive("IdleTimeBetweenReadsInMillis", idleTimeBetweenReadsInMillis); @@ -418,7 +547,6 @@ public class KinesisClientLibConfiguration { checkIsValuePositive("TaskBackoffTimeMillis", taskBackoffTimeMillis); checkIsValuePositive("MetricsBufferTimeMills", metricsBufferTimeMillis); checkIsValuePositive("MetricsMaxQueueSize", (long) metricsMaxQueueSize); - checkIsValuePositive("ShutdownGraceMillis", shutdownGraceMillis); checkIsRegionNameValid(regionName); this.applicationName = applicationName; this.tableName = applicationName; @@ -455,6 +583,7 @@ public class KinesisClientLibConfiguration { InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; + this.recordsFetcherFactory = recordsFetcherFactory; this.shutdownGraceMillis = shutdownGraceMillis; } @@ -1158,6 +1287,48 @@ public class KinesisClientLibConfiguration { return this; } + /** + * + * @param maxPendingProcessRecordsInput The max number of ProcessRecordsInput that can be stored in the cache before + * blocking + * @return this configuration object + */ + public KinesisClientLibConfiguration withMaxPendingProcessRecordsInput(final int maxPendingProcessRecordsInput) { + checkIsValuePositive("maxPendingProcessRecordsInput", maxPendingProcessRecordsInput); + this.recordsFetcherFactory.setMaxPendingProcessRecordsInput(maxPendingProcessRecordsInput); + return this; + } + + /** + * @param maxCacheByteSize Max byte size for the cache at any given point of time. After this threshold is crossed + * the KinesisDataFetcher will be blocked until the cache has more space available. + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withMaxCacheByteSize(final int maxCacheByteSize) { + checkIsValuePositive("maxCacheByteSize", maxCacheByteSize); + this.recordsFetcherFactory.setMaxByteSize(maxCacheByteSize); + return this; + } + + /** + * @param dataFetchingStrategy The strategy for fetching data from kinesis. + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withDataFetchingStrategy(String dataFetchingStrategy) { + this.recordsFetcherFactory.setDataFetchingStrategy(DataFetchingStrategy.valueOf(dataFetchingStrategy.toUpperCase())); + return this; + } + + /** + * @param maxRecordsCount The maximum number of records in the cache, accross all ProcessRecordInput objects + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withMaxRecordsCount(final int maxRecordsCount) { + checkIsValuePositive("maxRecordsCount", maxRecordsCount); + this.recordsFetcherFactory.setMaxRecordsCount(maxRecordsCount); + return this; + } + /** * @param timeoutInSeconds The timeout in seconds to wait for the MultiLangProtocol to wait for */ @@ -1174,4 +1345,14 @@ public class KinesisClientLibConfiguration { this.shutdownGraceMillis = shutdownGraceMillis; return this; } + + /** + * @param idleMillisBetweenCalls Idle time between 2 getcalls from the data fetcher. + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withIdleMillisBetweenCalls(long idleMillisBetweenCalls) { + checkIsValuePositive("IdleMillisBetweenCalls", idleMillisBetweenCalls); + this.recordsFetcherFactory.setIdleMillisBetweenCalls(idleMillisBetweenCalls); + return this; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java index dec0ac5e..c2ba9d15 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java @@ -1,33 +1,34 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; -import lombok.Data; +import java.util.Collections; +import java.util.Date; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import com.amazonaws.services.kinesis.model.GetRecordsResult; -import com.amazonaws.services.kinesis.model.ResourceNotFoundException; -import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.SentinelCheckpoint; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.MetricsCollectingKinesisProxyDecorator; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.ShardIteratorType; -import java.util.Date; -import java.util.function.Consumer; +import lombok.Data; /** * Used to get data from Amazon Kinesis. Tracks iterator state internally. @@ -49,8 +50,7 @@ class KinesisDataFetcher { */ public KinesisDataFetcher(IKinesisProxy kinesisProxy, ShardInfo shardInfo) { this.shardId = shardInfo.getShardId(); - this.kinesisProxy = - new MetricsCollectingKinesisProxyDecorator("KinesisDataFetcher", kinesisProxy, this.shardId); + this.kinesisProxy = new MetricsCollectingKinesisProxyDecorator("KinesisDataFetcher", kinesisProxy, this.shardId); } /** @@ -63,26 +63,24 @@ class KinesisDataFetcher { if (!isInitialized) { throw new IllegalArgumentException("KinesisDataFetcher.getRecords called before initialization."); } - - DataFetcherResult response; + if (nextIterator != null) { try { - response = new AdvancingResult(kinesisProxy.get(nextIterator, maxRecords)); + return new AdvancingResult(kinesisProxy.get(nextIterator, maxRecords)); } catch (ResourceNotFoundException e) { LOG.info("Caught ResourceNotFoundException when fetching records for shard " + shardId); - response = TERMINAL_RESULT; + return TERMINAL_RESULT; } } else { - response = TERMINAL_RESULT; + return TERMINAL_RESULT; } - - return response; } final DataFetcherResult TERMINAL_RESULT = new DataFetcherResult() { @Override public GetRecordsResult getResult() { - return null; + return new GetRecordsResult().withMillisBehindLatest(null).withRecords(Collections.emptyList()) + .withNextShardIterator(null); } @Override @@ -98,7 +96,7 @@ class KinesisDataFetcher { }; @Data - private class AdvancingResult implements DataFetcherResult { + class AdvancingResult implements DataFetcherResult { final GetRecordsResult result; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java new file mode 100644 index 00000000..5369c0f4 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java @@ -0,0 +1,223 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import java.time.Duration; +import java.time.Instant; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; + +import com.amazonaws.SdkClientException; +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; +import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingFactory; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.model.GetRecordsResult; + +import lombok.NonNull; +import lombok.extern.apachecommons.CommonsLog; +import org.apache.commons.lang.Validate; + +/** + * This is the prefetch caching class, this class spins up a thread if prefetching is enabled. That thread fetches the + * next set of records and stores it in the cache. The size of the cache is limited by setting + * maxPendingProcessRecordsInput i.e. the maximum number of GetRecordsResult that the cache can store, maxByteSize + * i.e. the byte size of the records stored in the cache and maxRecordsCount i.e. the max number of records that should + * be present in the cache across multiple GetRecordsResult object. If no data is available in the cache, the call from + * the record processor is blocked till records are retrieved from Kinesis. + */ +@CommonsLog +public class PrefetchGetRecordsCache implements GetRecordsCache { + LinkedBlockingQueue getRecordsResultQueue; + private int maxPendingProcessRecordsInput; + private int maxByteSize; + private int maxRecordsCount; + private final int maxRecordsPerCall; + private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private final ExecutorService executorService; + private final IMetricsFactory metricsFactory; + private final long idleMillisBetweenCalls; + private Instant lastSuccessfulCall; + private final DefaultGetRecordsCacheDaemon defaultGetRecordsCacheDaemon; + private PrefetchCounters prefetchCounters; + private boolean started = false; + private final String operation; + + /** + * Constructor for the PrefetchGetRecordsCache. This cache prefetches records from Kinesis and stores them in a + * LinkedBlockingQueue. + * + * @see com.amazonaws.services.kinesis.clientlibrary.lib.worker.PrefetchGetRecordsCache + * + * @param maxPendingProcessRecordsInput Max number of ProcessRecordsInput that can be held in the cache before + * blocking + * @param maxByteSize Max byte size of the queue before blocking next get records call + * @param maxRecordsCount Max number of records in the queue across all ProcessRecordInput objects + * @param maxRecordsPerCall Max records to be returned per call + * @param getRecordsRetrievalStrategy Retrieval strategy for the get records call + * @param executorService Executor service for the cache + * @param idleMillisBetweenCalls maximum time to wait before dispatching the next get records call + */ + public PrefetchGetRecordsCache(final int maxPendingProcessRecordsInput, final int maxByteSize, final int maxRecordsCount, + final int maxRecordsPerCall, + @NonNull final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, + @NonNull final ExecutorService executorService, + long idleMillisBetweenCalls, + @NonNull final IMetricsFactory metricsFactory, + @NonNull String operation) { + this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; + this.maxRecordsPerCall = maxRecordsPerCall; + this.maxPendingProcessRecordsInput = maxPendingProcessRecordsInput; + this.maxByteSize = maxByteSize; + this.maxRecordsCount = maxRecordsCount; + this.getRecordsResultQueue = new LinkedBlockingQueue<>(this.maxPendingProcessRecordsInput); + this.prefetchCounters = new PrefetchCounters(); + this.executorService = executorService; + this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory); + this.idleMillisBetweenCalls = idleMillisBetweenCalls; + this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon(); + Validate.notEmpty(operation, "Operation cannot be empty"); + this.operation = operation; + } + + @Override + public void start() { + if (executorService.isShutdown()) { + throw new IllegalStateException("ExecutorService has been shutdown."); + } + + if (!started) { + log.info("Starting prefetching thread."); + executorService.execute(defaultGetRecordsCacheDaemon); + } + started = true; + } + + @Override + public ProcessRecordsInput getNextResult() { + if (executorService.isShutdown()) { + throw new IllegalStateException("Shutdown has been called on the cache, can't accept new requests."); + } + + if (!started) { + throw new IllegalStateException("Cache has not been initialized, make sure to call start."); + } + ProcessRecordsInput result = null; + try { + result = getRecordsResultQueue.take().withCacheExitTime(Instant.now()); + prefetchCounters.removed(result); + } catch (InterruptedException e) { + log.error("Interrupted while getting records from the cache", e); + } + return result; + } + + @Override + public GetRecordsRetrievalStrategy getGetRecordsRetrievalStrategy() { + return getRecordsRetrievalStrategy; + } + + @Override + public void shutdown() { + defaultGetRecordsCacheDaemon.isShutdown = true; + executorService.shutdownNow(); + started = false; + } + + private class DefaultGetRecordsCacheDaemon implements Runnable { + volatile boolean isShutdown = false; + + @Override + public void run() { + while (!isShutdown) { + if (Thread.currentThread().isInterrupted()) { + log.warn("Prefetch thread was interrupted."); + break; + } + MetricsHelper.startScope(metricsFactory, operation); + if (prefetchCounters.shouldGetNewRecords()) { + try { + sleepBeforeNextCall(); + GetRecordsResult getRecordsResult = getRecordsRetrievalStrategy.getRecords(maxRecordsPerCall); + lastSuccessfulCall = Instant.now(); + ProcessRecordsInput processRecordsInput = new ProcessRecordsInput() + .withRecords(getRecordsResult.getRecords()) + .withMillisBehindLatest(getRecordsResult.getMillisBehindLatest()) + .withCacheEntryTime(lastSuccessfulCall); + getRecordsResultQueue.put(processRecordsInput); + prefetchCounters.added(processRecordsInput); + } catch (InterruptedException e) { + log.info("Thread was interrupted, indicating shutdown was called on the cache."); + } catch (SdkClientException e) { + log.error("Exception thrown while fetching records from Kinesis", e); + } catch (Throwable e) { + log.error("Unexpected exception was thrown. This could probably be an issue or a bug." + + " Please search for the exception/error online to check what is going on. If the " + + "issue persists or is a recurring problem, feel free to open an issue on, " + + "https://github.com/awslabs/amazon-kinesis-client.", e); + } finally { + MetricsHelper.endScope(); + } + } + } + callShutdownOnStrategy(); + } + + private void callShutdownOnStrategy() { + if (!getRecordsRetrievalStrategy.isShutdown()) { + getRecordsRetrievalStrategy.shutdown(); + } + } + + private void sleepBeforeNextCall() throws InterruptedException { + if (lastSuccessfulCall == null) { + return; + } + long timeSinceLastCall = Duration.between(lastSuccessfulCall, Instant.now()).abs().toMillis(); + if (timeSinceLastCall < idleMillisBetweenCalls) { + Thread.sleep(idleMillisBetweenCalls - timeSinceLastCall); + } + } + } + + private class PrefetchCounters { + private long size = 0; + private long byteSize = 0; + + public synchronized void added(final ProcessRecordsInput result) { + size += getSize(result); + byteSize += getByteSize(result); + } + + public synchronized void removed(final ProcessRecordsInput result) { + size -= getSize(result); + byteSize -= getByteSize(result); + } + + private long getSize(final ProcessRecordsInput result) { + return result.getRecords().size(); + } + + private long getByteSize(final ProcessRecordsInput result) { + return result.getRecords().stream().mapToLong(record -> record.getData().array().length).sum(); + } + + public synchronized boolean shouldGetNewRecords() { + return size < maxRecordsCount && byteSize < maxByteSize; + } + } + +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java index 90ac2c09..9aca832e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTask.java @@ -18,7 +18,6 @@ import java.math.BigInteger; import java.util.Collections; import java.util.List; import java.util.ListIterator; -import java.util.Optional; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -63,7 +62,7 @@ class ProcessTask implements ITask { private final Shard shard; private final ThrottlingReporter throttlingReporter; - private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private final GetRecordsCache getRecordsCache; /** * @param shardInfo @@ -78,17 +77,17 @@ class ProcessTask implements ITask { * Kinesis data fetcher (used to fetch records from Kinesis) * @param backoffTimeMillis * backoff time when catching exceptions - * @param getRecordsRetrievalStrategy + * @param getRecordsCache * The retrieval strategy for fetching records from kinesis */ public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, - GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { + GetRecordsCache getRecordsCache) { this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()), - getRecordsRetrievalStrategy); + getRecordsCache); } /** @@ -108,9 +107,9 @@ class ProcessTask implements ITask { * determines how throttling events should be reported in the log. */ public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor, - RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, - long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, - ThrottlingReporter throttlingReporter, GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { + RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher, + long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, + ThrottlingReporter throttlingReporter, GetRecordsCache getRecordsCache) { super(); this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; @@ -120,7 +119,7 @@ class ProcessTask implements ITask { this.backoffTimeMillis = backoffTimeMillis; this.throttlingReporter = throttlingReporter; IKinesisProxy kinesisProxy = this.streamConfig.getStreamProxy(); - this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; + this.getRecordsCache = getRecordsCache; // If skipShardSyncAtWorkerInitializationIfLeasesExist is set, we will not get the shard for // this ProcessTask. In this case, duplicate KPL user records in the event of resharding will // not be dropped during deaggregation of Amazon Kinesis records. This is only applicable if @@ -149,7 +148,6 @@ class ProcessTask implements ITask { scope.addDimension(MetricsHelper.SHARD_ID_DIMENSION_NAME, shardInfo.getShardId()); scope.addData(RECORDS_PROCESSED_METRIC, 0, StandardUnit.Count, MetricsLevel.SUMMARY); scope.addData(DATA_BYTES_PROCESSED_METRIC, 0, StandardUnit.Bytes, MetricsLevel.SUMMARY); - Exception exception = null; try { @@ -158,10 +156,10 @@ class ProcessTask implements ITask { return new TaskResult(null, true); } - final GetRecordsResult getRecordsResult = getRecordsResult(); + final ProcessRecordsInput processRecordsInput = getRecordsResult(); throttlingReporter.success(); - List records = getRecordsResult.getRecords(); - + List records = processRecordsInput.getRecords(); + if (!records.isEmpty()) { scope.addData(RECORDS_PROCESSED_METRIC, records.size(), StandardUnit.Count, MetricsLevel.SUMMARY); } else { @@ -175,7 +173,7 @@ class ProcessTask implements ITask { recordProcessorCheckpointer.getLargestPermittedCheckpointValue())); if (shouldCallProcessRecords(records)) { - callProcessRecords(getRecordsResult, records); + callProcessRecords(processRecordsInput, records); } } catch (ProvisionedThroughputExceededException pte) { throttlingReporter.throttled(); @@ -205,18 +203,18 @@ class ProcessTask implements ITask { /** * Dispatches a batch of records to the record processor, and handles any fallout from that. - * - * @param getRecordsResult + * + * @param input * the result of the last call to Kinesis * @param records * the records to be dispatched. It's possible the records have been adjusted by KPL deaggregation. */ - private void callProcessRecords(GetRecordsResult getRecordsResult, List records) { + private void callProcessRecords(ProcessRecordsInput input, List records) { LOG.debug("Calling application processRecords() with " + records.size() + " records from " + shardInfo.getShardId()); final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput().withRecords(records) .withCheckpointer(recordProcessorCheckpointer) - .withMillisBehindLatest(getRecordsResult.getMillisBehindLatest()); + .withMillisBehindLatest(input.getMillisBehindLatest()); final long recordProcessorStartTimeMillis = System.currentTimeMillis(); try { @@ -233,7 +231,7 @@ class ProcessTask implements ITask { /** * Whether we should call process records or not - * + * * @param records * the records returned from the call to Kinesis, and/or deaggregation * @return true if the set of records should be dispatched to the record process, false if they should not. @@ -244,7 +242,7 @@ class ProcessTask implements ITask { /** * Determines whether to deaggregate the given records, and if they are KPL records dispatches them to deaggregation - * + * * @param records * the records to deaggregate is deaggregation is required. * @return returns either the deaggregated records, or the original records @@ -267,7 +265,7 @@ class ProcessTask implements ITask { /** * Emits metrics, and sleeps if there are no records available - * + * * @param startTimeMillis * the time when the task started */ @@ -304,8 +302,8 @@ class ProcessTask implements ITask { * @return the largest extended sequence number among the retained records */ private ExtendedSequenceNumber filterAndGetMaxExtendedSequenceNumber(IMetricsScope scope, List records, - final ExtendedSequenceNumber lastCheckpointValue, - final ExtendedSequenceNumber lastLargestPermittedCheckpointValue) { + final ExtendedSequenceNumber lastCheckpointValue, + final ExtendedSequenceNumber lastLargestPermittedCheckpointValue) { ExtendedSequenceNumber largestExtendedSequenceNumber = lastLargestPermittedCheckpointValue; ListIterator recordIterator = records.listIterator(); while (recordIterator.hasNext()) { @@ -339,7 +337,7 @@ class ProcessTask implements ITask { * * @return list of data records from Kinesis */ - private GetRecordsResult getRecordsResult() { + private ProcessRecordsInput getRecordsResult() { try { return getRecordsResultAndRecordMillisBehindLatest(); } catch (ExpiredIteratorException e) { @@ -375,22 +373,17 @@ class ProcessTask implements ITask { * * @return list of data records from Kinesis */ - private GetRecordsResult getRecordsResultAndRecordMillisBehindLatest() { - final GetRecordsResult getRecordsResult = getRecordsRetrievalStrategy.getRecords(streamConfig.getMaxRecords()); + private ProcessRecordsInput getRecordsResultAndRecordMillisBehindLatest() { + final ProcessRecordsInput processRecordsInput = getRecordsCache.getNextResult(); - if (getRecordsResult == null) { - // Stream no longer exists - return new GetRecordsResult().withRecords(Collections.emptyList()); - } - - if (getRecordsResult.getMillisBehindLatest() != null) { + if (processRecordsInput.getMillisBehindLatest() != null) { MetricsHelper.getMetricsScope().addData(MILLIS_BEHIND_LATEST_METRIC, - getRecordsResult.getMillisBehindLatest(), + processRecordsInput.getMillisBehindLatest(), StandardUnit.Milliseconds, MetricsLevel.SUMMARY); } - return getRecordsResult; + return processRecordsInput; } -} +} \ No newline at end of file diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java new file mode 100644 index 00000000..be8316d7 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java @@ -0,0 +1,74 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; + +/** + * This factory is used to create the records fetcher to retrieve data from Kinesis for a given shard. + */ +public interface RecordsFetcherFactory { + /** + * Returns a GetRecordsCache to be used for retrieving records for a given shard. + * + * @param getRecordsRetrievalStrategy GetRecordsRetrievalStrategy to be used with the GetRecordsCache + * @param shardId ShardId of the shard that the fetcher will retrieve records for + * @param metricsFactory MetricsFactory used to create metricScope + * + * @return GetRecordsCache used to get records from Kinesis. + */ + GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory); + + /** + * Sets the maximum number of ProcessRecordsInput objects the GetRecordsCache can hold, before further requests are + * blocked. + * + * @param maxPendingProcessRecordsInput The maximum number of ProcessRecordsInput objects that the cache will accept + * before blocking. + */ + void setMaxPendingProcessRecordsInput(int maxPendingProcessRecordsInput); + + /** + * Sets the max byte size for the GetRecordsCache, before further requests are blocked. The byte size of the cache + * is the sum of byte size of all the ProcessRecordsInput objects in the cache at any point of time. + * + * @param maxByteSize The maximum byte size for the cache before blocking. + */ + void setMaxByteSize(int maxByteSize); + + /** + * Sets the max number of records for the GetRecordsCache can hold, before further requests are blocked. The records + * count is the sum of all records present in across all the ProcessRecordsInput objects in the cache at any point + * of time. + * + * @param maxRecordsCount The mximum number of records in the cache before blocking. + */ + void setMaxRecordsCount(int maxRecordsCount); + + /** + * Sets the dataFetchingStrategy to determine the type of GetRecordsCache to be used. + * + * @param dataFetchingStrategy Fetching strategy to be used + */ + void setDataFetchingStrategy(DataFetchingStrategy dataFetchingStrategy); + + /** + * Sets the maximum idle time between two get calls. + * + * @param idleMillisBetweenCalls Sleep millis between calls. + */ + void setIdleMillisBetweenCalls(long idleMillisBetweenCalls); + +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 4bbe1939..d7a5545e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -20,7 +20,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; -import lombok.Getter; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -32,6 +31,8 @@ import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.google.common.annotations.VisibleForTesting; +import lombok.Getter; + /** * Responsible for consuming data records of a (specified) shard. * The instance should be shutdown when we lose the primary responsibility for a shard. @@ -43,6 +44,7 @@ class ShardConsumer { private final StreamConfig streamConfig; private final IRecordProcessor recordProcessor; + private final KinesisClientLibConfiguration config; private final RecordProcessorCheckpointer recordProcessorCheckpointer; private final ExecutorService executorService; private final ShardInfo shardInfo; @@ -61,7 +63,7 @@ class ShardConsumer { private Future future; @Getter - private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private final GetRecordsCache getRecordsCache; private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher, Optional retryGetRecordsInSeconds, @@ -91,6 +93,7 @@ class ShardConsumer { * @param streamConfig Stream configuration to use * @param checkpoint Checkpoint tracker * @param recordProcessor Record processor used to process the data records for the shard + * @param config Kinesis library configuration * @param leaseManager Used to create leases for new shards * @param parentShardPollIntervalMillis Wait for this long if parent shards are not done (or we get an exception) * @param executorService ExecutorService used to execute process tasks for this shard @@ -99,19 +102,31 @@ class ShardConsumer { */ // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES ShardConsumer(ShardInfo shardInfo, - StreamConfig streamConfig, - ICheckpoint checkpoint, - IRecordProcessor recordProcessor, - ILeaseManager leaseManager, - long parentShardPollIntervalMillis, - boolean cleanupLeasesOfCompletedShards, - ExecutorService executorService, - IMetricsFactory metricsFactory, - long backoffTimeMillis, - boolean skipShardSyncAtWorkerInitializationIfLeasesExist) { - this(shardInfo, streamConfig, checkpoint,recordProcessor, leaseManager, parentShardPollIntervalMillis, - cleanupLeasesOfCompletedShards, executorService, metricsFactory, backoffTimeMillis, - skipShardSyncAtWorkerInitializationIfLeasesExist, Optional.empty(), Optional.empty()); + StreamConfig streamConfig, + ICheckpoint checkpoint, + IRecordProcessor recordProcessor, + ILeaseManager leaseManager, + long parentShardPollIntervalMillis, + boolean cleanupLeasesOfCompletedShards, + ExecutorService executorService, + IMetricsFactory metricsFactory, + long backoffTimeMillis, + boolean skipShardSyncAtWorkerInitializationIfLeasesExist, + KinesisClientLibConfiguration config) { + this(shardInfo, + streamConfig, + checkpoint, + recordProcessor, + leaseManager, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + backoffTimeMillis, + skipShardSyncAtWorkerInitializationIfLeasesExist, + Optional.empty(), + Optional.empty(), + config); } /** @@ -126,6 +141,7 @@ class ShardConsumer { * @param backoffTimeMillis backoff interval when we encounter exceptions * @param retryGetRecordsInSeconds time in seconds to wait before the worker retries to get a record. * @param maxGetRecordsThreadPool max number of threads in the getRecords thread pool. + * @param config Kinesis library configuration */ // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES ShardConsumer(ShardInfo shardInfo, @@ -140,26 +156,86 @@ class ShardConsumer { long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, Optional retryGetRecordsInSeconds, - Optional maxGetRecordsThreadPool) { - this.streamConfig = streamConfig; - this.recordProcessor = recordProcessor; - this.executorService = executorService; - this.shardInfo = shardInfo; - this.checkpoint = checkpoint; - this.recordProcessorCheckpointer = - new RecordProcessorCheckpointer(shardInfo, + Optional maxGetRecordsThreadPool, + KinesisClientLibConfiguration config) { + + this( + shardInfo, + streamConfig, + checkpoint, + recordProcessor, + new RecordProcessorCheckpointer( + shardInfo, checkpoint, - new SequenceNumberValidator(streamConfig.getStreamProxy(), + new SequenceNumberValidator( + streamConfig.getStreamProxy(), shardInfo.getShardId(), - streamConfig.shouldValidateSequenceNumberBeforeCheckpointing())); - this.dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); + streamConfig.shouldValidateSequenceNumberBeforeCheckpointing())), + leaseManager, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + backoffTimeMillis, + skipShardSyncAtWorkerInitializationIfLeasesExist, + new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo), + retryGetRecordsInSeconds, + maxGetRecordsThreadPool, + config + ); + } + + /** + * @param shardInfo Shard information + * @param streamConfig Stream Config to use + * @param checkpoint Checkpoint tracker + * @param recordProcessor Record processor used to process the data records for the shard + * @param recordProcessorCheckpointer RecordProcessorCheckpointer to use to checkpoint progress + * @param leaseManager Used to create leases for new shards + * @param parentShardPollIntervalMillis Wait for this long if parent shards are not done (or we get an exception) + * @param cleanupLeasesOfCompletedShards clean up the leases of completed shards + * @param executorService ExecutorService used to execute process tasks for this shard + * @param metricsFactory IMetricsFactory used to construct IMetricsScopes for this shard + * @param backoffTimeMillis backoff interval when we encounter exceptions + * @param skipShardSyncAtWorkerInitializationIfLeasesExist Skip sync at init if lease exists + * @param kinesisDataFetcher KinesisDataFetcher to fetch data from Kinesis streams. + * @param retryGetRecordsInSeconds time in seconds to wait before the worker retries to get a record + * @param maxGetRecordsThreadPool max number of threads in the getRecords thread pool + * @param config Kinesis library configuration + */ + ShardConsumer(ShardInfo shardInfo, + StreamConfig streamConfig, + ICheckpoint checkpoint, + IRecordProcessor recordProcessor, + RecordProcessorCheckpointer recordProcessorCheckpointer, + ILeaseManager leaseManager, + long parentShardPollIntervalMillis, + boolean cleanupLeasesOfCompletedShards, + ExecutorService executorService, + IMetricsFactory metricsFactory, + long backoffTimeMillis, + boolean skipShardSyncAtWorkerInitializationIfLeasesExist, + KinesisDataFetcher kinesisDataFetcher, + Optional retryGetRecordsInSeconds, + Optional maxGetRecordsThreadPool, + KinesisClientLibConfiguration config) { + this.shardInfo = shardInfo; + this.streamConfig = streamConfig; + this.checkpoint = checkpoint; + this.recordProcessor = recordProcessor; + this.recordProcessorCheckpointer = recordProcessorCheckpointer; this.leaseManager = leaseManager; - this.metricsFactory = metricsFactory; this.parentShardPollIntervalMillis = parentShardPollIntervalMillis; this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards; + this.executorService = executorService; + this.metricsFactory = metricsFactory; this.taskBackoffTimeMillis = backoffTimeMillis; this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist; - this.getRecordsRetrievalStrategy = makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo); + this.config = config; + this.dataFetcher = kinesisDataFetcher; + this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher( + makeStrategy(this.dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, this.shardInfo), + this.getShardInfo().getShardId(), this.metricsFactory); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index f56033a8..bd40d686 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -46,7 +46,7 @@ class ShutdownTask implements ITask { private final boolean cleanupLeasesOfCompletedShards; private final TaskType taskType = TaskType.SHUTDOWN; private final long backoffTimeMillis; - private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private final GetRecordsCache getRecordsCache; /** * Constructor. @@ -61,7 +61,7 @@ class ShutdownTask implements ITask { boolean cleanupLeasesOfCompletedShards, ILeaseManager leaseManager, long backoffTimeMillis, - GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { + GetRecordsCache getRecordsCache) { this.shardInfo = shardInfo; this.recordProcessor = recordProcessor; this.recordProcessorCheckpointer = recordProcessorCheckpointer; @@ -71,7 +71,7 @@ class ShutdownTask implements ITask { this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards; this.leaseManager = leaseManager; this.backoffTimeMillis = backoffTimeMillis; - this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; + this.getRecordsCache = getRecordsCache; } /* @@ -111,7 +111,7 @@ class ShutdownTask implements ITask { } } LOG.debug("Shutting down retrieval strategy."); - getRecordsRetrievalStrategy.shutdown(); + getRecordsCache.shutdown(); LOG.debug("Record processor completed shutdown() for shard " + shardInfo.getShardId()); } catch (Exception e) { applicationException = true; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java new file mode 100644 index 00000000..e6c9f3b0 --- /dev/null +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java @@ -0,0 +1,77 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import java.util.concurrent.Executors; + +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import lombok.extern.apachecommons.CommonsLog; + +@CommonsLog +public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory { + private final int maxRecords; + private int maxPendingProcessRecordsInput = 3; + private int maxByteSize = 8 * 1024 * 1024; + private int maxRecordsCount = 30000; + private long idleMillisBetweenCalls = 1500L; + private DataFetchingStrategy dataFetchingStrategy = DataFetchingStrategy.DEFAULT; + private IMetricsFactory metricsFactory; + + public SimpleRecordsFetcherFactory(int maxRecords) { + this.maxRecords = maxRecords; + } + + @Override + public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory) { + if(dataFetchingStrategy.equals(DataFetchingStrategy.DEFAULT)) { + return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy, idleMillisBetweenCalls); + } else { + return new PrefetchGetRecordsCache(maxPendingProcessRecordsInput, maxByteSize, maxRecordsCount, maxRecords, + getRecordsRetrievalStrategy, + Executors.newFixedThreadPool(1, new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("prefetch-cache-" + shardId + "-%04d") + .build()), + idleMillisBetweenCalls, + metricsFactory, + "ProcessTask"); + } + } + + @Override + public void setMaxPendingProcessRecordsInput(int maxPendingProcessRecordsInput){ + this.maxPendingProcessRecordsInput = maxPendingProcessRecordsInput; + } + + @Override + public void setMaxByteSize(int maxByteSize){ + this.maxByteSize = maxByteSize; + } + + @Override + public void setMaxRecordsCount(int maxRecordsCount) { + this.maxRecordsCount = maxRecordsCount; + } + + @Override + public void setDataFetchingStrategy(DataFetchingStrategy dataFetchingStrategy){ + this.dataFetchingStrategy = dataFetchingStrategy; + } + + public void setIdleMillisBetweenCalls(final long idleMillisBetweenCalls) { + this.idleMillisBetweenCalls = idleMillisBetweenCalls; + } +} diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index 3cfb9f2f..d2ea738d 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -73,6 +73,7 @@ public class Worker implements Runnable { private final String applicationName; private final IRecordProcessorFactory recordProcessorFactory; + private final KinesisClientLibConfiguration config; private final StreamConfig streamConfig; private final InitialPositionInStreamExtended initialPosition; private final ICheckpoint checkpointTracker; @@ -245,6 +246,7 @@ public class Worker implements Runnable { KinesisClientLibConfiguration config, AmazonKinesis kinesisClient, AmazonDynamoDB dynamoDBClient, IMetricsFactory metricsFactory, ExecutorService execService) { this(config.getApplicationName(), new V1ToV2RecordProcessorFactoryAdapter(recordProcessorFactory), + config, new StreamConfig( new KinesisProxyFactory(config.getKinesisCredentialsProvider(), kinesisClient) .getProxy(config.getStreamName()), @@ -306,6 +308,8 @@ public class Worker implements Runnable { * Name of the Kinesis application * @param recordProcessorFactory * Used to get record processor instances for processing data from shards + * @paran config + * Kinesis Library configuration * @param streamConfig * Stream configuration * @param initialPositionInStream @@ -333,24 +337,25 @@ public class Worker implements Runnable { */ // NOTE: This has package level access solely for testing // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES - Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, StreamConfig streamConfig, - InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, + Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, + StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization) { - this(applicationName, recordProcessorFactory, streamConfig, initialPositionInStream, parentShardPollIntervalMillis, + this(applicationName, recordProcessorFactory, config, streamConfig, initialPositionInStream, parentShardPollIntervalMillis, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, checkpoint, leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization, Optional.empty(), Optional.empty()); } - /** * @param applicationName * Name of the Kinesis application * @param recordProcessorFactory * Used to get record processor instances for processing data from shards + * @param config + * Kinesis Library Configuration * @param streamConfig * Stream configuration * @param initialPositionInStream @@ -382,7 +387,7 @@ public class Worker implements Runnable { */ // NOTE: This has package level access solely for testing // CHECKSTYLE:IGNORE ParameterNumber FOR NEXT 10 LINES - Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, StreamConfig streamConfig, + Worker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, @@ -391,6 +396,7 @@ public class Worker implements Runnable { Optional retryGetRecordsInSeconds, Optional maxGetRecordsThreadPool) { this.applicationName = applicationName; this.recordProcessorFactory = recordProcessorFactory; + this.config = config; this.streamConfig = streamConfig; this.initialPosition = initialPositionInStream; this.parentShardPollIntervalMillis = parentShardPollIntervalMillis; @@ -411,7 +417,6 @@ public class Worker implements Runnable { this.maxGetRecordsThreadPool = maxGetRecordsThreadPool; } - /** * @return the applicationName */ @@ -819,11 +824,11 @@ public class Worker implements Runnable { * * @param shardInfo * Kinesis shard info - * @param factory + * @param processorFactory * RecordProcessor factory * @return ShardConsumer for the shard */ - ShardConsumer createOrGetShardConsumer(ShardInfo shardInfo, IRecordProcessorFactory factory) { + ShardConsumer createOrGetShardConsumer(ShardInfo shardInfo, IRecordProcessorFactory processorFactory) { ShardConsumer consumer = shardInfoShardConsumerMap.get(shardInfo); // Instantiate a new consumer if we don't have one, or the one we // had was from an earlier @@ -832,20 +837,30 @@ public class Worker implements Runnable { // completely processed (shutdown reason terminate). if ((consumer == null) || (consumer.isShutdown() && consumer.getShutdownReason().equals(ShutdownReason.ZOMBIE))) { - consumer = buildConsumer(shardInfo, factory); + consumer = buildConsumer(shardInfo, processorFactory); shardInfoShardConsumerMap.put(shardInfo, consumer); wlog.infoForce("Created new shardConsumer for : " + shardInfo); } return consumer; } - protected ShardConsumer buildConsumer(ShardInfo shardInfo, IRecordProcessorFactory factory) { - IRecordProcessor recordProcessor = factory.createProcessor(); + protected ShardConsumer buildConsumer(ShardInfo shardInfo, IRecordProcessorFactory processorFactory) { + IRecordProcessor recordProcessor = processorFactory.createProcessor(); - return new ShardConsumer(shardInfo, streamConfig, checkpointTracker, recordProcessor, - leaseCoordinator.getLeaseManager(), parentShardPollIntervalMillis, cleanupLeasesUponShardCompletion, - executorService, metricsFactory, taskBackoffTimeMillis, - skipShardSyncAtWorkerInitializationIfLeasesExist, retryGetRecordsInSeconds, maxGetRecordsThreadPool); + return new ShardConsumer(shardInfo, + streamConfig, + checkpointTracker, + recordProcessor, + leaseCoordinator.getLeaseManager(), + parentShardPollIntervalMillis, + cleanupLeasesUponShardCompletion, + executorService, + metricsFactory, + taskBackoffTimeMillis, + skipShardSyncAtWorkerInitializationIfLeasesExist, + retryGetRecordsInSeconds, + maxGetRecordsThreadPool, + config); } @@ -1049,6 +1064,7 @@ public class Worker implements Runnable { public static class Builder { private IRecordProcessorFactory recordProcessorFactory; + private RecordsFetcherFactory recordsFetcherFactory; private KinesisClientLibConfiguration config; private AmazonKinesis kinesisClient; private AmazonDynamoDB dynamoDBClient; @@ -1244,6 +1260,7 @@ public class Worker implements Runnable { return new Worker(config.getApplicationName(), recordProcessorFactory, + config, new StreamConfig(new KinesisProxyFactory(config.getKinesisCredentialsProvider(), kinesisClient).getProxy(config.getStreamName()), config.getMaxRecords(), diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java index bd960c08..362af357 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/types/ProcessRecordsInput.java @@ -14,17 +14,25 @@ */ package com.amazonaws.services.kinesis.clientlibrary.types; +import java.time.Duration; +import java.time.Instant; import java.util.List; + import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer; import com.amazonaws.services.kinesis.model.Record; +import lombok.Getter; + /** * Container for the parameters to the IRecordProcessor's * {@link com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcessor#processRecords( * ProcessRecordsInput processRecordsInput) processRecords} method. */ public class ProcessRecordsInput { - + @Getter + private Instant cacheEntryTime; + @Getter + private Instant cacheExitTime; private List records; private IRecordProcessorCheckpointer checkpointer; private Long millisBehindLatest; @@ -96,4 +104,21 @@ public class ProcessRecordsInput { this.millisBehindLatest = millisBehindLatest; return this; } + + public ProcessRecordsInput withCacheEntryTime(Instant cacheEntryTime) { + this.cacheEntryTime = cacheEntryTime; + return this; + } + + public ProcessRecordsInput withCacheExitTime(Instant cacheExitTime) { + this.cacheExitTime = cacheExitTime; + return this; + } + + public Duration getTimeSpentInCache() { + if (cacheEntryTime == null || cacheExitTime == null) { + return Duration.ZERO; + } + return Duration.between(cacheEntryTime, cacheExitTime); + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java index 8e89204e..30b877e8 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java @@ -14,29 +14,6 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; -import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; -import com.amazonaws.services.kinesis.model.GetRecordsResult; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.junit.After; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; - -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionHandler; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; - - import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; @@ -50,6 +27,29 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + @RunWith(MockitoJUnitRunner.class) public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { @@ -125,7 +125,6 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { @Test @Ignore public void testInterrupted() throws InterruptedException, ExecutionException { - Future mockFuture = mock(Future.class); when(completionService.submit(any())).thenReturn(mockFuture); when(completionService.poll()).thenReturn(mockFuture); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java new file mode 100644 index 00000000..731c4653 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java @@ -0,0 +1,84 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.Record; + +/** + * Test class for the BlockingGetRecordsCache class. + */ +@RunWith(MockitoJUnitRunner.class) +public class BlockingGetRecordsCacheTest { + private static final int MAX_RECORDS_PER_COUNT = 10_000; + private static final long IDLE_MILLIS_BETWEEN_CALLS = 500L; + + @Mock + private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + @Mock + private GetRecordsResult getRecordsResult; + + private List records; + private BlockingGetRecordsCache blockingGetRecordsCache; + + @Before + public void setup() { + records = new ArrayList<>(); + blockingGetRecordsCache = new BlockingGetRecordsCache(MAX_RECORDS_PER_COUNT, getRecordsRetrievalStrategy, IDLE_MILLIS_BETWEEN_CALLS); + + when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_COUNT))).thenReturn(getRecordsResult); + when(getRecordsResult.getRecords()).thenReturn(records); + } + + @Test + public void testGetNextRecordsWithNoRecords() { + ProcessRecordsInput result = blockingGetRecordsCache.getNextResult(); + + assertEquals(result.getRecords(), records); + assertNull(result.getCacheEntryTime()); + assertNull(result.getCacheExitTime()); + assertEquals(result.getTimeSpentInCache(), Duration.ZERO); + } + + @Test + public void testGetNextRecordsWithRecords() { + Record record = new Record(); + records.add(record); + records.add(record); + records.add(record); + records.add(record); + + ProcessRecordsInput result = blockingGetRecordsCache.getNextResult(); + + assertEquals(result.getRecords(), records); + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java index 63f20a72..fa163ad2 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStatesTest.java @@ -55,6 +55,8 @@ public class ConsumerStatesTest { @Mock private IRecordProcessor recordProcessor; @Mock + private KinesisClientLibConfiguration config; + @Mock private RecordProcessorCheckpointer recordProcessorCheckpointer; @Mock private ExecutorService executorService; @@ -75,7 +77,7 @@ public class ConsumerStatesTest { @Mock private InitialPositionInStreamExtended initialPositionInStream; @Mock - private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private GetRecordsCache getRecordsCache; private long parentShardPollIntervalMillis = 0xCAFE; private boolean cleanupLeasesOfCompletedShards = true; @@ -98,7 +100,7 @@ public class ConsumerStatesTest { when(consumer.isCleanupLeasesOfCompletedShards()).thenReturn(cleanupLeasesOfCompletedShards); when(consumer.getTaskBackoffTimeMillis()).thenReturn(taskBackoffTimeMillis); when(consumer.getShutdownReason()).thenReturn(reason); - when(consumer.getGetRecordsRetrievalStrategy()).thenReturn(getRecordsRetrievalStrategy); + when(consumer.getGetRecordsCache()).thenReturn(getRecordsCache); } private static final Class> LEASE_MANAGER_CLASS = (Class>) (Class) ILeaseManager.class; @@ -207,6 +209,33 @@ public class ConsumerStatesTest { } + @Test + public void processingStateRecordsFetcher() { + + ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState(); + ITask task = state.createTask(consumer); + + assertThat(task, procTask(ShardInfo.class, "shardInfo", equalTo(shardInfo))); + assertThat(task, procTask(IRecordProcessor.class, "recordProcessor", equalTo(recordProcessor))); + assertThat(task, procTask(RecordProcessorCheckpointer.class, "recordProcessorCheckpointer", + equalTo(recordProcessorCheckpointer))); + assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher))); + assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig))); + assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis))); + + assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState())); + + assertThat(state.shutdownTransition(ShutdownReason.ZOMBIE), + equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState())); + assertThat(state.shutdownTransition(ShutdownReason.TERMINATE), + equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState())); + assertThat(state.shutdownTransition(ShutdownReason.REQUESTED), + equalTo(ShardConsumerState.SHUTDOWN_REQUESTED.getConsumerState())); + + assertThat(state.getState(), equalTo(ShardConsumerState.PROCESSING)); + assertThat(state.getTaskType(), equalTo(TaskType.PROCESS)); + } + @Test public void shutdownRequestState() { ConsumerState state = ShardConsumerState.SHUTDOWN_REQUESTED.getConsumerState(); @@ -313,7 +342,7 @@ public class ConsumerStatesTest { } static ReflectionPropertyMatcher shutdownTask(Class valueTypeClass, - String propertyName, Matcher matcher) { + String propertyName, Matcher matcher) { return taskWith(ShutdownTask.class, valueTypeClass, propertyName, matcher); } @@ -323,17 +352,17 @@ public class ConsumerStatesTest { } static ReflectionPropertyMatcher procTask(Class valueTypeClass, - String propertyName, Matcher matcher) { + String propertyName, Matcher matcher) { return taskWith(ProcessTask.class, valueTypeClass, propertyName, matcher); } static ReflectionPropertyMatcher initTask(Class valueTypeClass, - String propertyName, Matcher matcher) { + String propertyName, Matcher matcher) { return taskWith(InitializeTask.class, valueTypeClass, propertyName, matcher); } static ReflectionPropertyMatcher taskWith(Class taskTypeClass, - Class valueTypeClass, String propertyName, Matcher matcher) { + Class valueTypeClass, String propertyName, Matcher matcher) { return new ReflectionPropertyMatcher<>(taskTypeClass, valueTypeClass, matcher, propertyName); } @@ -346,7 +375,7 @@ public class ConsumerStatesTest { private final Field matchingField; private ReflectionPropertyMatcher(Class taskTypeClass, Class valueTypeClass, - Matcher matcher, String propertyName) { + Matcher matcher, String propertyName) { this.taskTypeClass = taskTypeClass; this.valueTypeClazz = valueTypeClass; this.matcher = matcher; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java index cfa8be10..177546db 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import junit.framework.Assert; +import java.util.Date; import org.junit.Test; import org.mockito.Mockito; @@ -35,7 +35,7 @@ import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorF import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; import com.google.common.collect.ImmutableSet; -import java.util.Date; +import junit.framework.Assert; public class KinesisClientLibConfigurationTest { private static final long INVALID_LONG = 0L; @@ -95,7 +95,8 @@ public class KinesisClientLibConfigurationTest { // Try each argument at one time. KinesisClientLibConfiguration config = null; long[] longValues = - { TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG }; + { TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, TEST_VALUE_LONG, + TEST_VALUE_LONG }; for (int i = 0; i < PARAMETER_COUNT; i++) { longValues[i] = INVALID_LONG; try { @@ -300,30 +301,30 @@ public class KinesisClientLibConfigurationTest { Mockito.mock(AWSCredentialsProvider.class); try { new KinesisClientLibConfiguration(TEST_STRING, - TEST_STRING, - TEST_STRING, - TEST_STRING, - null, - null, - null, - null, - TEST_VALUE_LONG, - TEST_STRING, - 3, - TEST_VALUE_LONG, - false, - TEST_VALUE_LONG, - TEST_VALUE_LONG, - true, - new ClientConfiguration(), - new ClientConfiguration(), - new ClientConfiguration(), - TEST_VALUE_LONG, - TEST_VALUE_LONG, - 1, - skipCheckpointValidationValue, - "abcd", - TEST_VALUE_LONG); + TEST_STRING, + TEST_STRING, + TEST_STRING, + null, + null, + null, + null, + TEST_VALUE_LONG, + TEST_STRING, + 3, + TEST_VALUE_LONG, + false, + TEST_VALUE_LONG, + TEST_VALUE_LONG, + true, + new ClientConfiguration(), + new ClientConfiguration(), + new ClientConfiguration(), + TEST_VALUE_LONG, + TEST_VALUE_LONG, + 1, + skipCheckpointValidationValue, + "abcd", + TEST_VALUE_LONG); Assert.fail("No expected Exception is thrown."); } catch(IllegalArgumentException e) { System.out.println(e.getMessage()); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java index 3cc8cb5a..6648b919 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java @@ -15,9 +15,10 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.collection.IsEmptyCollection.empty; import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; @@ -37,11 +38,10 @@ import java.util.List; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; -import com.amazonaws.services.kinesis.model.GetRecordsResult; -import com.amazonaws.services.kinesis.model.Record; -import com.amazonaws.services.kinesis.model.ResourceNotFoundException; -import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibException; import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint; import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.SentinelCheckpoint; @@ -50,9 +50,10 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.ShardIteratorType; /** * Unit tests for KinesisDataFetcher. @@ -73,7 +74,6 @@ public class KinesisDataFetcherTest { InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON); private static final InitialPositionInStreamExtended INITIAL_POSITION_AT_TIMESTAMP = InitialPositionInStreamExtended.newInitialPositionAtTimestamp(new Date(1000)); - ; /** * @throws java.lang.Exception @@ -208,6 +208,22 @@ public class KinesisDataFetcherTest { // Test shard has reached the end Assert.assertTrue("Shard should reach the end", dataFetcher.isShardEndReached()); } + + @Test + public void testNonNullGetRecords() { + String nextIterator = "TestIterator"; + int maxRecords = 100; + + KinesisProxy mockProxy = mock(KinesisProxy.class); + doThrow(new ResourceNotFoundException("Test Exception")).when(mockProxy).get(nextIterator, maxRecords); + + KinesisDataFetcher dataFetcher = new KinesisDataFetcher(mockProxy, SHARD_INFO); + dataFetcher.initialize(SentinelCheckpoint.LATEST.toString(), INITIAL_POSITION_LATEST); + + DataFetcherResult dataFetcherResult = dataFetcher.getRecords(maxRecords); + + assertThat(dataFetcherResult, notNullValue()); + } @Test public void testFetcherDoesNotAdvanceWithoutAccept() { @@ -227,9 +243,9 @@ public class KinesisDataFetcherTest { when(kinesisProxy.get(eq(NEXT_ITERATOR_TWO), anyInt())).thenReturn(finalResult); when(finalResult.getNextShardIterator()).thenReturn(null); - KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); - dataFetcher.initialize("TRIM_HORIZON", InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON)); + dataFetcher.initialize("TRIM_HORIZON", + InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.TRIM_HORIZON)); assertNoAdvance(dataFetcher, iteratorOneResults, INITIAL_ITERATOR); assertAdvanced(dataFetcher, iteratorOneResults, INITIAL_ITERATOR, NEXT_ITERATOR_ONE); @@ -248,14 +264,18 @@ public class KinesisDataFetcherTest { DataFetcherResult terminal = dataFetcher.getRecords(100); assertThat(terminal.isShardEnd(), equalTo(true)); - assertThat(terminal.getResult(), nullValue()); + assertThat(terminal.getResult(), notNullValue()); + GetRecordsResult terminalResult = terminal.getResult(); + assertThat(terminalResult.getRecords(), notNullValue()); + assertThat(terminalResult.getRecords(), empty()); + assertThat(terminalResult.getNextShardIterator(), nullValue()); assertThat(terminal, equalTo(dataFetcher.TERMINAL_RESULT)); verify(kinesisProxy, never()).get(anyString(), anyInt()); } - - private DataFetcherResult assertAdvanced(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, String previousValue, String nextValue) { + private DataFetcherResult assertAdvanced(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, + String previousValue, String nextValue) { DataFetcherResult acceptResult = dataFetcher.getRecords(100); assertThat(acceptResult.getResult(), equalTo(expectedResult)); @@ -273,7 +293,8 @@ public class KinesisDataFetcherTest { return acceptResult; } - private DataFetcherResult assertNoAdvance(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, String previousValue) { + private DataFetcherResult assertNoAdvance(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, + String previousValue) { assertThat(dataFetcher.getNextIterator(), equalTo(previousValue)); DataFetcherResult noAcceptResult = dataFetcher.getRecords(100); assertThat(noAcceptResult.getResult(), equalTo(expectedResult)); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java new file mode 100644 index 00000000..37d0e446 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java @@ -0,0 +1,199 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import com.amazonaws.services.kinesis.model.Record; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.Record; + +import lombok.extern.apachecommons.CommonsLog; + +/** + * These are the integration tests for the PrefetchGetRecordsCache class. + */ +@RunWith(MockitoJUnitRunner.class) +@CommonsLog +public class PrefetchGetRecordsCacheIntegrationTest { + private static final int MAX_SIZE = 3; + private static final int MAX_BYTE_SIZE = 5 * 1024 * 1024; + private static final int MAX_RECORDS_COUNT = 30_000; + private static final int MAX_RECORDS_PER_CALL = 10_000; + private static final long IDLE_MILLIS_BETWEEN_CALLS = 500L; + + private PrefetchGetRecordsCache getRecordsCache; + private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private KinesisDataFetcher dataFetcher; + private ExecutorService executorService; + private List records; + private String operation = "ProcessTask"; + + @Mock + private IKinesisProxy proxy; + + @Mock + private ShardInfo shardInfo; + + @Before + public void setup() { + records = new ArrayList<>(); + dataFetcher = new KinesisDataFetcherForTest(proxy, shardInfo); + getRecordsRetrievalStrategy = spy(new SynchronousGetRecordsRetrievalStrategy(dataFetcher)); + executorService = spy(Executors.newFixedThreadPool(1)); + + getRecordsCache = new PrefetchGetRecordsCache(MAX_SIZE, + MAX_BYTE_SIZE, + MAX_RECORDS_COUNT, + MAX_RECORDS_PER_CALL, + getRecordsRetrievalStrategy, + executorService, + IDLE_MILLIS_BETWEEN_CALLS, + new NullMetricsFactory(), + operation); + } + + @Test + public void testRollingCache() { + getRecordsCache.start(); + sleep(IDLE_MILLIS_BETWEEN_CALLS); + + ProcessRecordsInput processRecordsInput1 = getRecordsCache.getNextResult(); + + assertTrue(processRecordsInput1.getRecords().isEmpty()); + assertEquals(processRecordsInput1.getMillisBehindLatest(), new Long(1000)); + assertNotNull(processRecordsInput1.getCacheEntryTime()); + + ProcessRecordsInput processRecordsInput2 = getRecordsCache.getNextResult(); + + assertNotEquals(processRecordsInput1, processRecordsInput2); + } + + @Test + public void testFullCache() { + getRecordsCache.start(); + sleep(MAX_SIZE * IDLE_MILLIS_BETWEEN_CALLS); + + assertEquals(getRecordsCache.getRecordsResultQueue.size(), MAX_SIZE); + + ProcessRecordsInput processRecordsInput1 = getRecordsCache.getNextResult(); + ProcessRecordsInput processRecordsInput2 = getRecordsCache.getNextResult(); + + assertNotEquals(processRecordsInput1, processRecordsInput2); + } + + @Test + public void testDifferentShardCaches() { + ExecutorService executorService2 = spy(Executors.newFixedThreadPool(1)); + KinesisDataFetcher kinesisDataFetcher = spy(new KinesisDataFetcherForTest(proxy, shardInfo)); + GetRecordsRetrievalStrategy getRecordsRetrievalStrategy2 = spy(new AsynchronousGetRecordsRetrievalStrategy(kinesisDataFetcher, 5 , 5, "Test-shard")); + GetRecordsCache getRecordsCache2 = new PrefetchGetRecordsCache( + MAX_SIZE, + MAX_BYTE_SIZE, + MAX_RECORDS_COUNT, + MAX_RECORDS_PER_CALL, + getRecordsRetrievalStrategy2, + executorService2, + IDLE_MILLIS_BETWEEN_CALLS, + new NullMetricsFactory(), + operation); + + getRecordsCache.start(); + sleep(IDLE_MILLIS_BETWEEN_CALLS); + + Record record = mock(Record.class); + ByteBuffer byteBuffer = ByteBuffer.allocate(512 * 1024); + when(record.getData()).thenReturn(byteBuffer); + + records.add(record); + records.add(record); + records.add(record); + records.add(record); + getRecordsCache2.start(); + + sleep(IDLE_MILLIS_BETWEEN_CALLS); + + ProcessRecordsInput p1 = getRecordsCache.getNextResult(); + + ProcessRecordsInput p2 = getRecordsCache2.getNextResult(); + + assertNotEquals(p1, p2); + assertTrue(p1.getRecords().isEmpty()); + assertFalse(p2.getRecords().isEmpty()); + assertEquals(p2.getRecords().size(), records.size()); + + getRecordsCache2.shutdown(); + sleep(100L); + verify(executorService2).shutdownNow(); + verify(getRecordsRetrievalStrategy2).shutdown(); + } + + @After + public void shutdown() { + getRecordsCache.shutdown(); + sleep(100L); + verify(executorService).shutdownNow(); + verify(getRecordsRetrievalStrategy).shutdown(); + } + + private void sleep(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) {} + } + + private class KinesisDataFetcherForTest extends KinesisDataFetcher { + public KinesisDataFetcherForTest(final IKinesisProxy kinesisProxy, + final ShardInfo shardInfo) { + super(kinesisProxy, shardInfo); + } + + @Override + public DataFetcherResult getRecords(final int maxRecords) { + GetRecordsResult getRecordsResult = new GetRecordsResult(); + getRecordsResult.setRecords(new ArrayList<>(records)); + getRecordsResult.setMillisBehindLatest(1000L); + + return new AdvancingResult(getRecordsResult); + } + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java new file mode 100644 index 00000000..6091baa9 --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java @@ -0,0 +1,215 @@ +/* + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at + * + * http://aws.amazon.com/asl/ + * + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.stream.IntStream; + +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; + +import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.Record; + +/** + * Test class for the PrefetchGetRecordsCache class. + */ +@RunWith(MockitoJUnitRunner.class) +public class PrefetchGetRecordsCacheTest { + private static final int SIZE_512_KB = 512 * 1024; + private static final int SIZE_1_MB = 2 * SIZE_512_KB; + private static final int MAX_RECORDS_PER_CALL = 10000; + private static final int MAX_SIZE = 5; + private static final int MAX_RECORDS_COUNT = 15000; + private static final long IDLE_MILLIS_BETWEEN_CALLS = 0L; + + @Mock + private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + @Mock + private GetRecordsResult getRecordsResult; + @Mock + private Record record; + + private List records; + private ExecutorService executorService; + private LinkedBlockingQueue spyQueue; + private PrefetchGetRecordsCache getRecordsCache; + private String operation = "ProcessTask"; + + @Before + public void setup() { + executorService = spy(Executors.newFixedThreadPool(1)); + getRecordsCache = new PrefetchGetRecordsCache( + MAX_SIZE, + 3 * SIZE_1_MB, + MAX_RECORDS_COUNT, + MAX_RECORDS_PER_CALL, + getRecordsRetrievalStrategy, + executorService, + IDLE_MILLIS_BETWEEN_CALLS, + new NullMetricsFactory(), + operation); + spyQueue = spy(getRecordsCache.getRecordsResultQueue); + records = spy(new ArrayList<>()); + + when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_CALL))).thenReturn(getRecordsResult); + when(getRecordsResult.getRecords()).thenReturn(records); + } + + @Test + public void testGetRecords() { + when(records.size()).thenReturn(1000); + when(record.getData()).thenReturn(createByteBufferWithSize(SIZE_512_KB)); + + records.add(record); + records.add(record); + records.add(record); + records.add(record); + records.add(record); + + getRecordsCache.start(); + ProcessRecordsInput result = getRecordsCache.getNextResult(); + + assertEquals(result.getRecords(), records); + + verify(executorService).execute(any()); + verify(getRecordsRetrievalStrategy, atLeast(1)).getRecords(eq(MAX_RECORDS_PER_CALL)); + } + + @Test + public void testFullCacheByteSize() { + when(records.size()).thenReturn(500); + when(record.getData()).thenReturn(createByteBufferWithSize(SIZE_1_MB)); + + records.add(record); + + getRecordsCache.start(); + + // Sleep for a few seconds for the cache to fill up. + sleep(2000); + + verify(getRecordsRetrievalStrategy, times(3)).getRecords(eq(MAX_RECORDS_PER_CALL)); + assertEquals(spyQueue.size(), 3); + } + + @Test + public void testFullCacheRecordsCount() { + int recordsSize = 4500; + when(records.size()).thenReturn(recordsSize); + + getRecordsCache.start(); + + sleep(2000); + + int callRate = (int) Math.ceil((double) MAX_RECORDS_COUNT/recordsSize); + verify(getRecordsRetrievalStrategy, times(callRate)).getRecords(MAX_RECORDS_PER_CALL); + assertEquals(spyQueue.size(), callRate); + assertTrue(callRate < MAX_SIZE); + } + + @Test + public void testFullCacheSize() { + int recordsSize = 200; + when(records.size()).thenReturn(recordsSize); + + getRecordsCache.start(); + + // Sleep for a few seconds for the cache to fill up. + sleep(2000); + + verify(getRecordsRetrievalStrategy, times(MAX_SIZE + 1)).getRecords(eq(MAX_RECORDS_PER_CALL)); + assertEquals(spyQueue.size(), MAX_SIZE); + } + + @Test + public void testMultipleCacheCalls() { + int recordsSize = 20; + when(record.getData()).thenReturn(createByteBufferWithSize(1024)); + + IntStream.range(0, recordsSize).forEach(i -> records.add(record)); + + getRecordsCache.start(); + ProcessRecordsInput processRecordsInput = getRecordsCache.getNextResult(); + + verify(executorService).execute(any()); + assertEquals(processRecordsInput.getRecords(), records); + assertNotNull(processRecordsInput.getCacheEntryTime()); + assertNotNull(processRecordsInput.getCacheExitTime()); + + sleep(2000); + + ProcessRecordsInput processRecordsInput2 = getRecordsCache.getNextResult(); + assertNotEquals(processRecordsInput, processRecordsInput2); + assertEquals(processRecordsInput2.getRecords(), records); + assertNotEquals(processRecordsInput2.getTimeSpentInCache(), Duration.ZERO); + + assertTrue(spyQueue.size() <= MAX_SIZE); + } + + @Test(expected = IllegalStateException.class) + public void testGetNextRecordsWithoutStarting() { + verify(executorService, times(0)).execute(any()); + getRecordsCache.getNextResult(); + } + + @Test(expected = IllegalStateException.class) + public void testCallAfterShutdown() { + when(executorService.isShutdown()).thenReturn(true); + getRecordsCache.getNextResult(); + } + + @After + public void shutdown() { + getRecordsCache.shutdown(); + verify(executorService).shutdownNow(); + } + + private void sleep(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) {} + } + + private ByteBuffer createByteBufferWithSize(int size) { + ByteBuffer byteBuffer = ByteBuffer.allocate(size); + byteBuffer.put(new byte[size]); + return byteBuffer; + } +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java index b24bf3ec..94d0918e 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ProcessTaskTest.java @@ -18,8 +18,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.never; @@ -49,7 +48,6 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber import com.amazonaws.services.kinesis.clientlibrary.types.Messages.AggregatedRecord; import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; -import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.Record; import com.google.protobuf.ByteString; @@ -77,7 +75,7 @@ public class ProcessTaskTest { @Mock private ThrottlingReporter throttlingReporter; @Mock - private GetRecordsRetrievalStrategy mockGetRecordsRetrievalStrategy; + private GetRecordsCache getRecordsCache; private List processedRecords; private ExtendedSequenceNumber newLargestPermittedCheckpointValue; @@ -95,32 +93,39 @@ public class ProcessTaskTest { INITIAL_POSITION_LATEST); final ShardInfo shardInfo = new ShardInfo(shardId, null, null, null); processTask = new ProcessTask( - shardInfo, config, mockRecordProcessor, mockCheckpointer, mockDataFetcher, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, throttlingReporter, mockGetRecordsRetrievalStrategy); + shardInfo, + config, + mockRecordProcessor, + mockCheckpointer, + mockDataFetcher, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + throttlingReporter, + getRecordsCache); } @Test public void testProcessTaskWithProvisionedThroughputExceededException() { // Set data fetcher to throw exception doReturn(false).when(mockDataFetcher).isShardEndReached(); - doThrow(new ProvisionedThroughputExceededException("Test Exception")).when(mockGetRecordsRetrievalStrategy) - .getRecords(maxRecords); + doThrow(new ProvisionedThroughputExceededException("Test Exception")).when(getRecordsCache) + .getNextResult(); TaskResult result = processTask.call(); verify(throttlingReporter).throttled(); verify(throttlingReporter, never()).success(); - verify(mockGetRecordsRetrievalStrategy).getRecords(eq(maxRecords)); + verify(getRecordsCache).getNextResult(); assertTrue("Result should contain ProvisionedThroughputExceededException", result.getException() instanceof ProvisionedThroughputExceededException); } @Test public void testProcessTaskWithNonExistentStream() { - // Data fetcher returns a null Result when the stream does not exist - doReturn(null).when(mockGetRecordsRetrievalStrategy).getRecords(maxRecords); + // Data fetcher returns a null Result ` the stream does not exist + doReturn(new ProcessRecordsInput().withRecords(Collections.emptyList()).withMillisBehindLatest((long) 0)).when(getRecordsCache).getNextResult(); TaskResult result = processTask.call(); - verify(mockGetRecordsRetrievalStrategy).getRecords(eq(maxRecords)); + verify(getRecordsCache).getNextResult(); assertNull("Task should not throw an exception", result.getException()); } @@ -304,14 +309,13 @@ public class ProcessTaskTest { private void testWithRecords(List records, ExtendedSequenceNumber lastCheckpointValue, ExtendedSequenceNumber largestPermittedCheckpointValue) { - when(mockGetRecordsRetrievalStrategy.getRecords(anyInt())).thenReturn( - new GetRecordsResult().withRecords(records)); + when(getRecordsCache.getNextResult()).thenReturn(new ProcessRecordsInput().withRecords(records).withMillisBehindLatest((long) 1000 * 50)); when(mockCheckpointer.getLastCheckpointValue()).thenReturn(lastCheckpointValue); when(mockCheckpointer.getLargestPermittedCheckpointValue()).thenReturn(largestPermittedCheckpointValue); processTask.call(); verify(throttlingReporter).success(); verify(throttlingReporter, never()).throttled(); - verify(mockGetRecordsRetrievalStrategy).getRecords(anyInt()); + verify(getRecordsCache).getNextResult(); ArgumentCaptor priCaptor = ArgumentCaptor.forClass(ProcessRecordsInput.class); verify(mockRecordProcessor).processRecords(priCaptor.capture()); processedRecords = priCaptor.getValue().getRecords(); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java new file mode 100644 index 00000000..912804da --- /dev/null +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java @@ -0,0 +1,44 @@ +package com.amazonaws.services.kinesis.clientlibrary.lib.worker; + +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +public class RecordsFetcherFactoryTest { + private String shardId = "TestShard"; + private RecordsFetcherFactory recordsFetcherFactory; + + @Mock + private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + + @Mock + private IMetricsFactory metricsFactory; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + recordsFetcherFactory = new SimpleRecordsFetcherFactory(1); + } + + @Test + public void createDefaultRecordsFetcherTest() { + GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId, + metricsFactory); + assertThat(recordsCache, instanceOf(BlockingGetRecordsCache.class)); + } + + @Test + public void createPrefetchRecordsFetcherTest() { + recordsFetcherFactory.setDataFetchingStrategy(DataFetchingStrategy.PREFETCH_CACHED); + GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId, + metricsFactory); + assertThat(recordsCache, instanceOf(PrefetchGetRecordsCache.class)); + } + +} diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index a3f786a6..0bd2f31a 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -20,7 +20,6 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; @@ -52,6 +51,7 @@ import org.apache.commons.logging.LogFactory; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -95,10 +95,16 @@ public class ShardConsumerTest { // Use Executors.newFixedThreadPool since it returns ThreadPoolExecutor, which is // ... a non-final public class, and so can be mocked and spied. private final ExecutorService executorService = Executors.newFixedThreadPool(1); - + private final int maxRecords = 500; + private RecordsFetcherFactory recordsFetcherFactory; + + private GetRecordsCache getRecordsCache; + @Mock private IRecordProcessor processor; @Mock + private KinesisClientLibConfiguration config; + @Mock private IKinesisProxy streamProxy; @Mock private ILeaseManager leaseManager; @@ -107,6 +113,14 @@ public class ShardConsumerTest { @Mock private ShutdownNotification shutdownNotification; + @Before + public void setup() { + getRecordsCache = null; + + recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory(maxRecords)); + when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); + } + /** * Test method to verify consumer stays in INITIALIZING state when InitializationTask fails. */ @@ -137,8 +151,9 @@ public class ShardConsumerTest { executorService, metricsFactory, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); - + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + config); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // initialize Thread.sleep(50L); @@ -154,7 +169,6 @@ public class ShardConsumerTest { assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); } - /** * Test method to verify consumer stays in INITIALIZING state when InitializationTask fails. */ @@ -185,7 +199,8 @@ public class ShardConsumerTest { spyExecutorService, metricsFactory, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + config); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // initialize @@ -226,7 +241,8 @@ public class ShardConsumerTest { executorService, metricsFactory, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + config); final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = null; @@ -299,7 +315,6 @@ public class ShardConsumerTest { ICheckpoint checkpoint = new InMemoryCheckpointImpl(startSeqNum.toString()); checkpoint.setCheckpoint(streamShardId, ExtendedSequenceNumber.TRIM_HORIZON, testConcurrencyToken); when(leaseManager.getLease(anyString())).thenReturn(null); - TestStreamlet processor = new TestStreamlet(); StreamConfig streamConfig = @@ -310,18 +325,41 @@ public class ShardConsumerTest { skipCheckpointValidationValue, INITIAL_POSITION_LATEST); ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, null); + + RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( + shardInfo, + checkpoint, + new SequenceNumberValidator( + streamConfig.getStreamProxy(), + shardInfo.getShardId(), + streamConfig.shouldValidateSequenceNumberBeforeCheckpointing() + ) + ); + + KinesisDataFetcher dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); + + getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, + new SynchronousGetRecordsRetrievalStrategy(dataFetcher), + 0L)); + when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); + ShardConsumer consumer = new ShardConsumer(shardInfo, streamConfig, checkpoint, processor, + recordProcessorCheckpointer, leaseManager, parentShardPollIntervalMillis, cleanupLeasesOfCompletedShards, executorService, metricsFactory, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + dataFetcher, + Optional.empty(), + Optional.empty(), + config); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // check on parent shards @@ -330,6 +368,7 @@ public class ShardConsumerTest { assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); consumer.consumeShard(); // initialize processor.getInitializeLatch().await(5, TimeUnit.SECONDS); + verify(getRecordsCache).start(); // We expect to process all records in numRecs calls for (int i = 0; i < numRecs;) { @@ -342,6 +381,8 @@ public class ShardConsumerTest { } Thread.sleep(50L); } + + verify(getRecordsCache, times(5)).getNextResult(); assertThat(processor.getShutdownReason(), nullValue()); consumer.notifyShutdownRequested(shutdownNotification); @@ -365,6 +406,8 @@ public class ShardConsumerTest { verify(shutdownNotification, atLeastOnce()).shutdownComplete(); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE))); assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.ZOMBIE))); + + verify(getRecordsCache).shutdown(); executorService.shutdown(); executorService.awaitTermination(60, TimeUnit.SECONDS); @@ -401,7 +444,6 @@ public class ShardConsumerTest { ICheckpoint checkpoint = new InMemoryCheckpointImpl(startSeqNum.toString()); checkpoint.setCheckpoint(streamShardId, ExtendedSequenceNumber.AT_TIMESTAMP, testConcurrencyToken); when(leaseManager.getLease(anyString())).thenReturn(null); - TestStreamlet processor = new TestStreamlet(); StreamConfig streamConfig = @@ -413,18 +455,41 @@ public class ShardConsumerTest { atTimestamp); ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, ExtendedSequenceNumber.TRIM_HORIZON); + + RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( + shardInfo, + checkpoint, + new SequenceNumberValidator( + streamConfig.getStreamProxy(), + shardInfo.getShardId(), + streamConfig.shouldValidateSequenceNumberBeforeCheckpointing() + ) + ); + + KinesisDataFetcher dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); + + getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, + new SynchronousGetRecordsRetrievalStrategy(dataFetcher), + 0L)); + when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); + ShardConsumer consumer = new ShardConsumer(shardInfo, streamConfig, checkpoint, processor, + recordProcessorCheckpointer, leaseManager, parentShardPollIntervalMillis, cleanupLeasesOfCompletedShards, executorService, metricsFactory, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + dataFetcher, + Optional.empty(), + Optional.empty(), + config); assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); consumer.consumeShard(); // check on parent shards @@ -433,6 +498,8 @@ public class ShardConsumerTest { assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); consumer.consumeShard(); // initialize Thread.sleep(50L); + + verify(getRecordsCache).start(); // We expect to process all records in numRecs calls for (int i = 0; i < numRecs;) { @@ -445,6 +512,8 @@ public class ShardConsumerTest { } Thread.sleep(50L); } + + verify(getRecordsCache, times(4)).getNextResult(); assertThat(processor.getShutdownReason(), nullValue()); consumer.beginShutdown(); @@ -457,8 +526,11 @@ public class ShardConsumerTest { executorService.shutdown(); executorService.awaitTermination(60, TimeUnit.SECONDS); + verify(getRecordsCache).shutdown(); + String iterator = fileBasedProxy.getIterator(streamShardId, timestamp); List expectedRecords = toUserRecords(fileBasedProxy.get(iterator, numRecs).getRecords()); + verifyConsumedRecords(expectedRecords, processor.getProcessedRecords()); assertEquals(4, processor.getProcessedRecords().size()); file.delete(); @@ -486,11 +558,15 @@ public class ShardConsumerTest { executorService, metricsFactory, taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + config); + + GetRecordsCache getRecordsCache = spy(consumer.getGetRecordsCache()); final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = new ExtendedSequenceNumber("999"); when(leaseManager.getLease(anyString())).thenReturn(null); + when(config.getRecordsFetcherFactory()).thenReturn(new SimpleRecordsFetcherFactory(2)); when(checkpoint.getCheckpointObject(anyString())).thenReturn( new Checkpoint(checkpointSequenceNumber, pendingCheckpointSequenceNumber)); @@ -535,9 +611,11 @@ public class ShardConsumerTest { taskBackoffTimeMillis, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, Optional.empty(), - Optional.empty()); + Optional.empty(), + config); - assertEquals(shardConsumer.getGetRecordsRetrievalStrategy().getClass(), SynchronousGetRecordsRetrievalStrategy.class); + assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(), + SynchronousGetRecordsRetrievalStrategy.class); } @Test @@ -563,9 +641,11 @@ public class ShardConsumerTest { taskBackoffTimeMillis, KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, Optional.of(1), - Optional.of(2)); + Optional.of(2), + config); - assertEquals(shardConsumer.getGetRecordsRetrievalStrategy().getClass(), AsynchronousGetRecordsRetrievalStrategy.class); + assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(), + AsynchronousGetRecordsRetrievalStrategy.class); } //@formatter:off (gets the formatting wrong) diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 5d91c698..17a53137 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -59,7 +59,7 @@ public class ShutdownTaskTest { IRecordProcessor defaultRecordProcessor = new TestStreamlet(); @Mock - private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; + private GetRecordsCache getRecordsCache; /** * @throws java.lang.Exception @@ -80,7 +80,7 @@ public class ShutdownTaskTest { */ @Before public void setUp() throws Exception { - doNothing().when(getRecordsRetrievalStrategy).shutdown(); + doNothing().when(getRecordsCache).shutdown(); } /** @@ -109,7 +109,7 @@ public class ShutdownTaskTest { cleanupLeasesOfCompletedShards, leaseManager, TASK_BACKOFF_TIME_MILLIS, - getRecordsRetrievalStrategy); + getRecordsCache); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof IllegalArgumentException); @@ -135,11 +135,11 @@ public class ShutdownTaskTest { cleanupLeasesOfCompletedShards, leaseManager, TASK_BACKOFF_TIME_MILLIS, - getRecordsRetrievalStrategy); + getRecordsCache); TaskResult result = task.call(); Assert.assertNotNull(result.getException()); Assert.assertTrue(result.getException() instanceof KinesisClientLibIOException); - verify(getRecordsRetrievalStrategy).shutdown(); + verify(getRecordsCache).shutdown(); } /** @@ -147,7 +147,7 @@ public class ShutdownTaskTest { */ @Test public final void testGetTaskType() { - ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, null, 0, getRecordsRetrievalStrategy); + ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, null, 0, getRecordsCache); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index 5913bf0d..a8856a0b 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -21,10 +21,20 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.eq; import static org.mockito.Matchers.same; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import java.io.File; import java.lang.Thread.State; @@ -60,6 +70,7 @@ import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; import org.hamcrest.TypeSafeMatcher; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Matchers; @@ -129,6 +140,9 @@ public class WorkerTest { private static final String KINESIS_SHARD_ID_FORMAT = "kinesis-0-0-%d"; private static final String CONCURRENCY_TOKEN_FORMAT = "testToken-%d"; + + private RecordsFetcherFactory recordsFetcherFactory; + private KinesisClientLibConfiguration config; @Mock private KinesisClientLibLeaseCoordinator leaseCoordinator; @@ -154,6 +168,13 @@ public class WorkerTest { private Future taskFuture; @Mock private TaskResult taskResult; + + @Before + public void setup() { + config = spy(new KinesisClientLibConfiguration("app", null, null, null)); + recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory(500)); + when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); + } // CHECKSTYLE:IGNORE AnonInnerLengthCheck FOR NEXT 50 LINES private static final com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory SAMPLE_RECORD_PROCESSOR_FACTORY = @@ -195,14 +216,13 @@ public class WorkerTest { /** - * Test method for {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker#getApplicationName()}. + * Test method for {@link Worker#getApplicationName()}. */ @Test public final void testGetStageName() { final String stageName = "testStageName"; - final KinesisClientLibConfiguration clientConfig = - new KinesisClientLibConfiguration(stageName, null, null, null); - Worker worker = new Worker(v1RecordProcessorFactory, clientConfig); + config = new KinesisClientLibConfiguration(stageName, null, null, null); + Worker worker = new Worker(v1RecordProcessorFactory, config); Assert.assertEquals(stageName, worker.getApplicationName()); } @@ -210,6 +230,7 @@ public class WorkerTest { public final void testCreateOrGetShardConsumer() { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; + config = new KinesisClientLibConfiguration(stageName, null, null, null); IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; @@ -228,7 +249,9 @@ public class WorkerTest { Worker worker = new Worker(stageName, - streamletFactory, streamConfig, INITIAL_POSITION_LATEST, + streamletFactory, + config, + streamConfig, INITIAL_POSITION_LATEST, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupLeasesUponShardCompletion, @@ -275,10 +298,22 @@ public class WorkerTest { when(leaseCoordinator.getCurrentAssignments()).thenReturn(initialState).thenReturn(firstCheckpoint) .thenReturn(secondCheckpoint); - Worker worker = new Worker(stageName, streamletFactory, streamConfig, INITIAL_POSITION_LATEST, - parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupLeasesUponShardCompletion, checkpoint, - leaseCoordinator, execService, nullMetricsFactory, taskBackoffTimeMillis, failoverTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, shardPrioritization); + Worker worker = new Worker(stageName, + streamletFactory, + config, + streamConfig, + INITIAL_POSITION_LATEST, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + checkpoint, + leaseCoordinator, + execService, + nullMetricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + shardPrioritization); Worker workerSpy = spy(worker); @@ -314,6 +349,7 @@ public class WorkerTest { public final void testCleanupShardConsumers() { final String stageName = "testStageName"; IRecordProcessorFactory streamletFactory = SAMPLE_RECORD_PROCESSOR_FACTORY_V2; + config = new KinesisClientLibConfiguration(stageName, null, null, null); IKinesisProxy proxy = null; ICheckpoint checkpoint = null; int maxRecords = 1; @@ -332,7 +368,9 @@ public class WorkerTest { Worker worker = new Worker(stageName, - streamletFactory, streamConfig, INITIAL_POSITION_LATEST, + streamletFactory, + config, + streamConfig, INITIAL_POSITION_LATEST, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupLeasesUponShardCompletion, @@ -371,6 +409,7 @@ public class WorkerTest { public final void testInitializationFailureWithRetries() { String stageName = "testInitializationWorker"; IRecordProcessorFactory recordProcessorFactory = new TestStreamletFactory(null, null); + config = new KinesisClientLibConfiguration(stageName, null, null, null); int count = 0; when(proxy.getShardList()).thenThrow(new RuntimeException(Integer.toString(count++))); int maxRecords = 2; @@ -386,6 +425,7 @@ public class WorkerTest { Worker worker = new Worker(stageName, recordProcessorFactory, + config, streamConfig, INITIAL_POSITION_TRIM_HORIZON, shardPollInterval, shardSyncIntervalMillis, @@ -437,7 +477,7 @@ public class WorkerTest { /** * Runs worker with threadPoolSize < numShards - * Test method for {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker#run()}. + * Test method for {@link Worker#run()}. */ @Test public final void testOneSplitShard2Threads() throws Exception { @@ -448,12 +488,12 @@ public class WorkerTest { KinesisClientLease lease = ShardSyncer.newKCLLease(shardList.get(0)); lease.setCheckpoint(new ExtendedSequenceNumber("2")); initialLeases.add(lease); - runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard); + runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config); } /** * Runs worker with threadPoolSize < numShards - * Test method for {@link com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker#run()}. + * Test method for {@link Worker#run()}. */ @Test public final void testOneSplitShard2ThreadsWithCallsForEmptyRecords() throws Exception { @@ -465,7 +505,10 @@ public class WorkerTest { lease.setCheckpoint(new ExtendedSequenceNumber("2")); initialLeases.add(lease); boolean callProcessRecordsForEmptyRecordList = true; - runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard); + RecordsFetcherFactory recordsFetcherFactory = new SimpleRecordsFetcherFactory(500); + recordsFetcherFactory.setIdleMillisBetweenCalls(0L); + when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); + runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config); } @Test @@ -490,7 +533,8 @@ public class WorkerTest { 10, kinesisProxy, v2RecordProcessorFactory, executorService, - cwMetricsFactory); + cwMetricsFactory, + config); // Give some time for thread to run. workerStarted.await(); @@ -526,7 +570,8 @@ public class WorkerTest { 10, kinesisProxy, v2RecordProcessorFactory, executorService, - cwMetricsFactory); + cwMetricsFactory, + config); // Give some time for thread to run. workerStarted.await(); @@ -572,6 +617,12 @@ public class WorkerTest { return null; } }).when(v2RecordProcessor).processRecords(any(ProcessRecordsInput.class)); + + RecordsFetcherFactory recordsFetcherFactory = mock(RecordsFetcherFactory.class); + GetRecordsCache getRecordsCache = mock(GetRecordsCache.class); + when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); + when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); + when(getRecordsCache.getNextResult()).thenReturn(new ProcessRecordsInput().withRecords(Collections.emptyList()).withMillisBehindLatest(0L)); WorkerThread workerThread = runWorker(shardList, initialLeases, @@ -581,7 +632,8 @@ public class WorkerTest { fileBasedProxy, v2RecordProcessorFactory, executorService, - nullMetricsFactory); + nullMetricsFactory, + config); // Only sleep for time that is required. processRecordsLatch.await(); @@ -672,7 +724,8 @@ public class WorkerTest { fileBasedProxy, v2RecordProcessorFactory, executorService, - nullMetricsFactory); + nullMetricsFactory, + config); // Only sleep for time that is required. processRecordsLatch.await(); @@ -742,10 +795,22 @@ public class WorkerTest { when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization); + Worker worker = new Worker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization); when(executorService.submit(Matchers.> any())) .thenAnswer(new ShutdownHandlingAnswer(taskFuture)); @@ -816,7 +881,7 @@ public class WorkerTest { IRecordProcessor processor = mock(IRecordProcessor.class); when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, streamConfig, + Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, config, streamConfig, INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) { @@ -888,10 +953,22 @@ public class WorkerTest { when(coordinator.startGracefulShutdown(any(Callable.class))).thenReturn(gracefulShutdownFuture); - Worker worker = new InjectableWorker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization) { + Worker worker = new InjectableWorker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization) { @Override void postConstruct() { this.gracefulShutdownCoordinator = coordinator; @@ -950,10 +1027,22 @@ public class WorkerTest { when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization); + Worker worker = new Worker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization); when(executorService.submit(Matchers.> any())) .thenAnswer(new ShutdownHandlingAnswer(taskFuture)); @@ -1020,10 +1109,22 @@ public class WorkerTest { IRecordProcessor processor = mock(IRecordProcessor.class); when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization); + Worker worker = new Worker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization); when(executorService.submit(Matchers.> any())) .thenAnswer(new ShutdownHandlingAnswer(taskFuture)); @@ -1121,10 +1222,22 @@ public class WorkerTest { IRecordProcessor processor = mock(IRecordProcessor.class); when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization); + Worker worker = new Worker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization); when(executorService.submit(Matchers.> any())) .thenAnswer(new ShutdownHandlingAnswer(taskFuture)); @@ -1226,10 +1339,22 @@ public class WorkerTest { IRecordProcessor processor = mock(IRecordProcessor.class); when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization); + Worker worker = new Worker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization); when(executorService.submit(Matchers.> any())) .thenAnswer(new ShutdownHandlingAnswer(taskFuture)); @@ -1298,10 +1423,22 @@ public class WorkerTest { IRecordProcessor processor = mock(IRecordProcessor.class); when(recordProcessorFactory.createProcessor()).thenReturn(processor); - Worker worker = new Worker("testRequestShutdown", recordProcessorFactory, streamConfig, - INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, - cleanupLeasesUponShardCompletion, leaseCoordinator, leaseCoordinator, executorService, metricsFactory, - taskBackoffTimeMillis, failoverTimeMillis, false, shardPrioritization); + Worker worker = new Worker("testRequestShutdown", + recordProcessorFactory, + config, + streamConfig, + INITIAL_POSITION_TRIM_HORIZON, + parentShardPollIntervalMillis, + shardSyncIntervalMillis, + cleanupLeasesUponShardCompletion, + leaseCoordinator, + leaseCoordinator, + executorService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + false, + shardPrioritization); when(executorService.submit(Matchers.> any())) .thenAnswer(new ShutdownHandlingAnswer(taskFuture)); @@ -1336,16 +1473,29 @@ public class WorkerTest { private abstract class InjectableWorker extends Worker { InjectableWorker(String applicationName, IRecordProcessorFactory recordProcessorFactory, - StreamConfig streamConfig, InitialPositionInStreamExtended initialPositionInStream, + KinesisClientLibConfiguration config, StreamConfig streamConfig, + InitialPositionInStreamExtended initialPositionInStream, long parentShardPollIntervalMillis, long shardSyncIdleTimeMillis, boolean cleanupLeasesUponShardCompletion, ICheckpoint checkpoint, KinesisClientLibLeaseCoordinator leaseCoordinator, ExecutorService execService, IMetricsFactory metricsFactory, long taskBackoffTimeMillis, long failoverTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist, ShardPrioritization shardPrioritization) { - super(applicationName, recordProcessorFactory, streamConfig, initialPositionInStream, - parentShardPollIntervalMillis, shardSyncIdleTimeMillis, cleanupLeasesUponShardCompletion, - checkpoint, leaseCoordinator, execService, metricsFactory, taskBackoffTimeMillis, - failoverTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist, shardPrioritization); + super(applicationName, + recordProcessorFactory, + config, + streamConfig, + initialPositionInStream, + parentShardPollIntervalMillis, + shardSyncIdleTimeMillis, + cleanupLeasesUponShardCompletion, + checkpoint, + leaseCoordinator, + execService, + metricsFactory, + taskBackoffTimeMillis, + failoverTimeMillis, + skipShardSyncAtWorkerInitializationIfLeasesExist, + shardPrioritization); postConstruct(); } @@ -1578,14 +1728,15 @@ public class WorkerTest { lease.setCheckpoint(ExtendedSequenceNumber.AT_TIMESTAMP); initialLeases.add(lease); } - runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard); + runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config); } private void runAndTestWorker(List shardList, - int threadPoolSize, - List initialLeases, - boolean callProcessRecordsForEmptyRecordList, - int numberOfRecordsPerShard) throws Exception { + int threadPoolSize, + List initialLeases, + boolean callProcessRecordsForEmptyRecordList, + int numberOfRecordsPerShard, + KinesisClientLibConfiguration clientConfig) throws Exception { File file = KinesisLocalFileDataCreator.generateTempDataFile(shardList, numberOfRecordsPerShard, "unitTestWT001"); IKinesisProxy fileBasedProxy = new KinesisLocalFileProxy(file.getAbsolutePath()); @@ -1594,10 +1745,10 @@ public class WorkerTest { TestStreamletFactory recordProcessorFactory = new TestStreamletFactory(recordCounter, shardSequenceVerifier); ExecutorService executorService = Executors.newFixedThreadPool(threadPoolSize); - + WorkerThread workerThread = runWorker( shardList, initialLeases, callProcessRecordsForEmptyRecordList, failoverTimeMillis, - numberOfRecordsPerShard, fileBasedProxy, recordProcessorFactory, executorService, nullMetricsFactory); + numberOfRecordsPerShard, fileBasedProxy, recordProcessorFactory, executorService, nullMetricsFactory, clientConfig); // TestStreamlet will release the semaphore once for every record it processes recordCounter.acquire(numberOfRecordsPerShard * shardList.size()); @@ -1614,14 +1765,15 @@ public class WorkerTest { } private WorkerThread runWorker(List shardList, - List initialLeases, - boolean callProcessRecordsForEmptyRecordList, - long failoverTimeMillis, - int numberOfRecordsPerShard, - IKinesisProxy kinesisProxy, - IRecordProcessorFactory recordProcessorFactory, - ExecutorService executorService, - IMetricsFactory metricsFactory) throws Exception { + List initialLeases, + boolean callProcessRecordsForEmptyRecordList, + long failoverTimeMillis, + int numberOfRecordsPerShard, + IKinesisProxy kinesisProxy, + IRecordProcessorFactory recordProcessorFactory, + ExecutorService executorService, + IMetricsFactory metricsFactory, + KinesisClientLibConfiguration clientConfig) throws Exception { final String stageName = "testStageName"; final int maxRecords = 2; @@ -1649,10 +1801,11 @@ public class WorkerTest { idleTimeInMilliseconds, callProcessRecordsForEmptyRecordList, skipCheckpointValidationValue, InitialPositionInStreamExtended.newInitialPositionAtTimestamp(timestamp)); - + Worker worker = new Worker(stageName, recordProcessorFactory, + clientConfig, streamConfig, INITIAL_POSITION_TRIM_HORIZON, parentShardPollIntervalMillis, shardSyncIntervalMillis, From 44437f13610aeff177650a3d6e15545f56395177 Mon Sep 17 00:00:00 2001 From: jmooreoliva Date: Wed, 18 Oct 2017 14:16:54 -0700 Subject: [PATCH 14/35] Shutdown that throws an exception will be retried. (#238) Without this change a transient error on shutdown with reason terminate prevents child shards from starting. --- .../lib/worker/ShardConsumer.java | 2 +- .../lib/worker/ShardConsumerTest.java | 124 ++++++++++++++++++ 2 files changed, 125 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index d7a5545e..33187ad7 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -406,7 +406,7 @@ class ShardConsumer { if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.TERMINATE); } - if (isShutdownRequested()) { + if (isShutdownRequested() && taskOutcome != TaskOutcome.FAILURE) { currentState = currentState.shutdownTransition(shutdownReason); } else if (taskOutcome == TaskOutcome.SUCCESSFUL) { if (currentState.getTaskType() == currentTask.getTaskType()) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 0bd2f31a..1858e07c 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.ListIterator; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -66,12 +67,14 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProx import com.amazonaws.services.kinesis.clientlibrary.proxies.util.KinesisLocalFileDataCreator; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput; +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; /** @@ -418,6 +421,127 @@ public class ShardConsumerTest { file.delete(); } + private static final class TransientShutdownErrorTestStreamlet extends TestStreamlet { + private final CountDownLatch errorShutdownLatch = new CountDownLatch(1); + + @Override + public void shutdown(ShutdownInput input) { + ShutdownReason reason = input.getShutdownReason(); + if (reason.equals(ShutdownReason.TERMINATE) && errorShutdownLatch.getCount() > 0) { + errorShutdownLatch.countDown(); + throw new RuntimeException("test"); + } else { + super.shutdown(input); + } + } + } + + /** + * Test method for {@link ShardConsumer#consumeShard()} that ensures a transient error thrown from the record + * processor's shutdown method with reason terminate will be retried. + */ + @Test + public final void testConsumeShardWithTransientTerminateError() throws Exception { + int numRecs = 10; + BigInteger startSeqNum = BigInteger.ONE; + String streamShardId = "kinesis-0-0"; + String testConcurrencyToken = "testToken"; + List shardList = KinesisLocalFileDataCreator.createShardList(1, "kinesis-0-", startSeqNum); + // Close the shard so that shutdown is called with reason terminate + shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( + KinesisLocalFileProxy.MAX_SEQUENCE_NUMBER.subtract(BigInteger.ONE).toString()); + File file = KinesisLocalFileDataCreator.generateTempDataFile(shardList, numRecs, "unitTestSCT002"); + + IKinesisProxy fileBasedProxy = new KinesisLocalFileProxy(file.getAbsolutePath()); + + final int maxRecords = 2; + final int idleTimeMS = 0; // keep unit tests fast + ICheckpoint checkpoint = new InMemoryCheckpointImpl(startSeqNum.toString()); + checkpoint.setCheckpoint(streamShardId, ExtendedSequenceNumber.TRIM_HORIZON, testConcurrencyToken); + when(leaseManager.getLease(anyString())).thenReturn(null); + + TransientShutdownErrorTestStreamlet processor = new TransientShutdownErrorTestStreamlet(); + + StreamConfig streamConfig = + new StreamConfig(fileBasedProxy, + maxRecords, + idleTimeMS, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, null); + ShardConsumer consumer = + new ShardConsumer(shardInfo, + streamConfig, + checkpoint, + processor, + leaseManager, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); + + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); + consumer.consumeShard(); // check on parent shards + Thread.sleep(50L); + consumer.consumeShard(); // start initialization + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); + consumer.consumeShard(); // initialize + processor.getInitializeLatch().await(5, TimeUnit.SECONDS); + + // We expect to process all records in numRecs calls + for (int i = 0; i < numRecs;) { + boolean newTaskSubmitted = consumer.consumeShard(); + if (newTaskSubmitted) { + LOG.debug("New processing task was submitted, call # " + i); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING))); + // CHECKSTYLE:IGNORE ModifiedControlVariable FOR NEXT 1 LINES + i += maxRecords; + } + Thread.sleep(50L); + } + + // Consume shards until shutdown terminate is called and it has thrown an exception + for (int i = 0; i < 100; i++) { + consumer.consumeShard(); + if (processor.errorShutdownLatch.await(50, TimeUnit.MILLISECONDS)) { + break; + } + } + assertEquals(0, processor.errorShutdownLatch.getCount()); + + // Wait for a retry of shutdown terminate that should succeed + for (int i = 0; i < 100; i++) { + consumer.consumeShard(); + if (processor.getShutdownLatch().await(50, TimeUnit.MILLISECONDS)) { + break; + } + } + assertEquals(0, processor.getShutdownLatch().getCount()); + + // Wait for shutdown complete now that terminate shutdown is successful + for (int i = 0; i < 100; i++) { + consumer.consumeShard(); + if (consumer.getCurrentState() == ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE) { + break; + } + Thread.sleep(50L); + } + assertThat(consumer.getCurrentState(), equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE)); + + assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.TERMINATE))); + + executorService.shutdown(); + executorService.awaitTermination(60, TimeUnit.SECONDS); + + String iterator = fileBasedProxy.getIterator(streamShardId, ShardIteratorType.TRIM_HORIZON.toString()); + List expectedRecords = toUserRecords(fileBasedProxy.get(iterator, numRecs).getRecords()); + verifyConsumedRecords(expectedRecords, processor.getProcessedRecords()); + file.delete(); + } + /** * Test method for {@link ShardConsumer#consumeShard()} that starts from initial position of type AT_TIMESTAMP. */ From 8ed6c81ceabb5a124a644c914a1e4b407374cfb3 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Thu, 19 Oct 2017 11:40:44 -0700 Subject: [PATCH 15/35] Revert "Shutdown that throws an exception will be retried. (#238)" (#251) This reverts commit 44437f13610aeff177650a3d6e15545f56395177. Reverted due to Build Failures --- .../lib/worker/ShardConsumer.java | 2 +- .../lib/worker/ShardConsumerTest.java | 124 ------------------ 2 files changed, 1 insertion(+), 125 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 33187ad7..d7a5545e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -406,7 +406,7 @@ class ShardConsumer { if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.TERMINATE); } - if (isShutdownRequested() && taskOutcome != TaskOutcome.FAILURE) { + if (isShutdownRequested()) { currentState = currentState.shutdownTransition(shutdownReason); } else if (taskOutcome == TaskOutcome.SUCCESSFUL) { if (currentState.getTaskType() == currentTask.getTaskType()) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 1858e07c..0bd2f31a 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -40,7 +40,6 @@ import java.util.List; import java.util.ListIterator; import java.util.Objects; import java.util.Optional; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -67,14 +66,12 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProx import com.amazonaws.services.kinesis.clientlibrary.proxies.util.KinesisLocalFileDataCreator; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput; -import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.Record; -import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; /** @@ -421,127 +418,6 @@ public class ShardConsumerTest { file.delete(); } - private static final class TransientShutdownErrorTestStreamlet extends TestStreamlet { - private final CountDownLatch errorShutdownLatch = new CountDownLatch(1); - - @Override - public void shutdown(ShutdownInput input) { - ShutdownReason reason = input.getShutdownReason(); - if (reason.equals(ShutdownReason.TERMINATE) && errorShutdownLatch.getCount() > 0) { - errorShutdownLatch.countDown(); - throw new RuntimeException("test"); - } else { - super.shutdown(input); - } - } - } - - /** - * Test method for {@link ShardConsumer#consumeShard()} that ensures a transient error thrown from the record - * processor's shutdown method with reason terminate will be retried. - */ - @Test - public final void testConsumeShardWithTransientTerminateError() throws Exception { - int numRecs = 10; - BigInteger startSeqNum = BigInteger.ONE; - String streamShardId = "kinesis-0-0"; - String testConcurrencyToken = "testToken"; - List shardList = KinesisLocalFileDataCreator.createShardList(1, "kinesis-0-", startSeqNum); - // Close the shard so that shutdown is called with reason terminate - shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( - KinesisLocalFileProxy.MAX_SEQUENCE_NUMBER.subtract(BigInteger.ONE).toString()); - File file = KinesisLocalFileDataCreator.generateTempDataFile(shardList, numRecs, "unitTestSCT002"); - - IKinesisProxy fileBasedProxy = new KinesisLocalFileProxy(file.getAbsolutePath()); - - final int maxRecords = 2; - final int idleTimeMS = 0; // keep unit tests fast - ICheckpoint checkpoint = new InMemoryCheckpointImpl(startSeqNum.toString()); - checkpoint.setCheckpoint(streamShardId, ExtendedSequenceNumber.TRIM_HORIZON, testConcurrencyToken); - when(leaseManager.getLease(anyString())).thenReturn(null); - - TransientShutdownErrorTestStreamlet processor = new TransientShutdownErrorTestStreamlet(); - - StreamConfig streamConfig = - new StreamConfig(fileBasedProxy, - maxRecords, - idleTimeMS, - callProcessRecordsForEmptyRecordList, - skipCheckpointValidationValue, INITIAL_POSITION_LATEST); - - ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, null); - ShardConsumer consumer = - new ShardConsumer(shardInfo, - streamConfig, - checkpoint, - processor, - leaseManager, - parentShardPollIntervalMillis, - cleanupLeasesOfCompletedShards, - executorService, - metricsFactory, - taskBackoffTimeMillis, - KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST); - - assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); - consumer.consumeShard(); // check on parent shards - Thread.sleep(50L); - consumer.consumeShard(); // start initialization - assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); - consumer.consumeShard(); // initialize - processor.getInitializeLatch().await(5, TimeUnit.SECONDS); - - // We expect to process all records in numRecs calls - for (int i = 0; i < numRecs;) { - boolean newTaskSubmitted = consumer.consumeShard(); - if (newTaskSubmitted) { - LOG.debug("New processing task was submitted, call # " + i); - assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING))); - // CHECKSTYLE:IGNORE ModifiedControlVariable FOR NEXT 1 LINES - i += maxRecords; - } - Thread.sleep(50L); - } - - // Consume shards until shutdown terminate is called and it has thrown an exception - for (int i = 0; i < 100; i++) { - consumer.consumeShard(); - if (processor.errorShutdownLatch.await(50, TimeUnit.MILLISECONDS)) { - break; - } - } - assertEquals(0, processor.errorShutdownLatch.getCount()); - - // Wait for a retry of shutdown terminate that should succeed - for (int i = 0; i < 100; i++) { - consumer.consumeShard(); - if (processor.getShutdownLatch().await(50, TimeUnit.MILLISECONDS)) { - break; - } - } - assertEquals(0, processor.getShutdownLatch().getCount()); - - // Wait for shutdown complete now that terminate shutdown is successful - for (int i = 0; i < 100; i++) { - consumer.consumeShard(); - if (consumer.getCurrentState() == ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE) { - break; - } - Thread.sleep(50L); - } - assertThat(consumer.getCurrentState(), equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE)); - - assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.TERMINATE))); - - executorService.shutdown(); - executorService.awaitTermination(60, TimeUnit.SECONDS); - - String iterator = fileBasedProxy.getIterator(streamShardId, ShardIteratorType.TRIM_HORIZON.toString()); - List expectedRecords = toUserRecords(fileBasedProxy.get(iterator, numRecs).getRecords()); - verifyConsumedRecords(expectedRecords, processor.getProcessedRecords()); - file.delete(); - } - /** * Test method for {@link ShardConsumer#consumeShard()} that starts from initial position of type AT_TIMESTAMP. */ From 046e160e24036fc237d732b1122383c912d1b813 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Mon, 23 Oct 2017 10:16:03 -0700 Subject: [PATCH 16/35] Block Fetch Thread When Retrieval Should be Paused (#252) Block the fetching thread when the queue is considered to be full. This ensures that the thread won't spin the CPU when it can't retrieve more records or bytes. --- .../lib/worker/PrefetchGetRecordsCache.java | 30 ++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java index 5369c0f4..06e77c8c 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java @@ -20,6 +20,8 @@ import java.time.Instant; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; +import org.apache.commons.lang.Validate; + import com.amazonaws.SdkClientException; import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; @@ -29,7 +31,6 @@ import com.amazonaws.services.kinesis.model.GetRecordsResult; import lombok.NonNull; import lombok.extern.apachecommons.CommonsLog; -import org.apache.commons.lang.Validate; /** * This is the prefetch caching class, this class spins up a thread if prefetching is enabled. That thread fetches the @@ -171,6 +172,16 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { } finally { MetricsHelper.endScope(); } + } else { + // + // Consumer isn't ready to receive new records will allow prefetch counters to pause + // + try { + prefetchCounters.waitForConsumer(); + } catch (InterruptedException ie) { + log.info("Thread was interrupted while waiting for the consumer. " + + "Shutdown has probably been started"); + } } } callShutdownOnStrategy(); @@ -205,6 +216,7 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { public synchronized void removed(final ProcessRecordsInput result) { size -= getSize(result); byteSize -= getByteSize(result); + this.notifyAll(); } private long getSize(final ProcessRecordsInput result) { @@ -214,10 +226,26 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { private long getByteSize(final ProcessRecordsInput result) { return result.getRecords().stream().mapToLong(record -> record.getData().array().length).sum(); } + + public synchronized void waitForConsumer() throws InterruptedException { + if (!shouldGetNewRecords()) { + log.debug("Queue is full waiting for consumer for " + idleMillisBetweenCalls + " ms"); + this.wait(idleMillisBetweenCalls); + } + } public synchronized boolean shouldGetNewRecords() { + if (log.isDebugEnabled()) { + log.debug("Current Prefetch Counter States: " + this.toString()); + } return size < maxRecordsCount && byteSize < maxByteSize; } + + @Override + public String toString() { + return String.format("{ Requests: %d, Records: %d, Bytes: %d }", getRecordsResultQueue.size(), size, + byteSize); + } } } From 90d4cb78c5b8bf3e3b3cf634b94d8dba0ae9bec9 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Mon, 23 Oct 2017 10:16:25 -0700 Subject: [PATCH 17/35] Update the AWS SDK Version to 1.11.218 (#253) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0e40b5ab..3e9b2c68 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ - 1.11.198 + 1.11.218 1.0.392 libsqlite4java ${project.build.directory}/test-lib From cc7e329e2fe1781073abc6d8892204be47480263 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Mon, 23 Oct 2017 12:30:08 -0700 Subject: [PATCH 18/35] Release Note for Release 1.8.6 of the Amazon Kinesis Client for Java (#254) --- META-INF/MANIFEST.MF | 4 ++-- README.md | 16 ++++++++++++++++ pom.xml | 2 +- .../worker/KinesisClientLibConfiguration.java | 2 +- 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index 146a18fe..c48417b0 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,9 +2,9 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.5 +Bundle-Version: 1.8.6 Bundle-Vendor: Amazon Technologies, Inc -Bundle-RequiredExecutionEnvironment: JavaSE-1.7 +Bundle-RequiredExecutionEnvironment: JavaSE-1.8 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", org.apache.commons.logging;bundle-version="1.1.3";visibility:=reexport, com.fasterxml.jackson.core.jackson-databind;bundle-version="2.5.3", diff --git a/README.md b/README.md index 8191254a..ee1b7f78 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,22 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.6 +* Add prefetching of records from Kinesis + Prefetching will retrieve and queue additional records from Kinesis while the application is processing existing records. + Prefetching can be enabled by setting [`dataFetchingStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1317) to `PREFETCH_CACHED`. Once enabled an additional fetching thread will be started to retrieve records from Kinesis. Retrieved records will be held in a queue until the application is ready to process them. + Pre-fetching supports the following configuration values: + + | Name | Default | Description | + | ---- | ------- | ----------- | + | [`dataFetchingStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1317) | `DEFAULT` | Which data fetching strategy to use | + | [`maxPendingProcessRecordsInput`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1296) | 3 | The maximum number of process records input that can be queued | + | [`maxCacheByteSize`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1307) | 8 MiB | The maximum number of bytes that can be queued | + | [`maxRecordsCount`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1326) | 30,000 | The maximum number of records that can be queued | + | [`idleMillisBetweenCalls`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1353) | 1,500 ms | The amount of time to wait between calls to Kinesis | + + * [PR #246](https://github.com/awslabs/amazon-kinesis-client/pull/246) + ### Release 1.8.5 (September 26, 2017) * Only advance the shard iterator for the accepted response. This fixes a race condition in the `KinesisDataFetcher` when it's being used to make asynchronous requests. The shard iterator is now only advanced when the retriever calls `DataFetcherResult#accept()`. diff --git a/pom.xml b/pom.xml index 3e9b2c68..46bba4ca 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.6-SNAPSHOT + 1.8.6 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index b0ac6bfd..da2610be 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.5"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.6"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From 73426bd73347ecae0e19626d79fa014d3dc225e7 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Tue, 24 Oct 2017 09:13:19 -0700 Subject: [PATCH 19/35] Don't Sleep for During Retrieval for the BlockingGetRecordsCache The BlockingGetRecordsCache shouldn't sleep when retrieving records as backoff is provided in other parts of the ShardConumer. --- .../lib/worker/BlockingGetRecordsCache.java | 29 ++----------------- .../worker/SimpleRecordsFetcherFactory.java | 3 +- .../worker/BlockingGetRecordsCacheTest.java | 3 +- .../lib/worker/ShardConsumerTest.java | 6 ++-- 4 files changed, 6 insertions(+), 35 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java index d9fc011e..021d886b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCache.java @@ -31,15 +31,11 @@ import lombok.extern.apachecommons.CommonsLog; public class BlockingGetRecordsCache implements GetRecordsCache { private final int maxRecordsPerCall; private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; - private final long idleMillisBetweenCalls; - private Instant lastSuccessfulCall; public BlockingGetRecordsCache(final int maxRecordsPerCall, - final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, - final long idleMillisBetweenCalls) { + final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) { this.maxRecordsPerCall = maxRecordsPerCall; this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; - this.idleMillisBetweenCalls = idleMillisBetweenCalls; } @Override @@ -51,33 +47,12 @@ public class BlockingGetRecordsCache implements GetRecordsCache { @Override public ProcessRecordsInput getNextResult() { - sleepBeforeNextCall(); GetRecordsResult getRecordsResult = getRecordsRetrievalStrategy.getRecords(maxRecordsPerCall); - lastSuccessfulCall = Instant.now(); - ProcessRecordsInput processRecordsInput = new ProcessRecordsInput() + return new ProcessRecordsInput() .withRecords(getRecordsResult.getRecords()) .withMillisBehindLatest(getRecordsResult.getMillisBehindLatest()); - return processRecordsInput; } - private void sleepBeforeNextCall() { - if (!Thread.interrupted()) { - if (lastSuccessfulCall == null) { - return; - } - long timeSinceLastCall = Duration.between(lastSuccessfulCall, Instant.now()).abs().toMillis(); - if (timeSinceLastCall < idleMillisBetweenCalls) { - try { - Thread.sleep(idleMillisBetweenCalls - timeSinceLastCall); - } catch (InterruptedException e) { - log.info("Thread was interrupted, indicating that shutdown was called."); - } - } - } else { - log.info("Thread has been interrupted, indicating that it is in the shutdown phase."); - } - } - @Override public GetRecordsRetrievalStrategy getGetRecordsRetrievalStrategy() { return getRecordsRetrievalStrategy; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java index e6c9f3b0..44c93e7b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java @@ -28,7 +28,6 @@ public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory { private int maxRecordsCount = 30000; private long idleMillisBetweenCalls = 1500L; private DataFetchingStrategy dataFetchingStrategy = DataFetchingStrategy.DEFAULT; - private IMetricsFactory metricsFactory; public SimpleRecordsFetcherFactory(int maxRecords) { this.maxRecords = maxRecords; @@ -37,7 +36,7 @@ public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory { @Override public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory) { if(dataFetchingStrategy.equals(DataFetchingStrategy.DEFAULT)) { - return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy, idleMillisBetweenCalls); + return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy); } else { return new PrefetchGetRecordsCache(maxPendingProcessRecordsInput, maxByteSize, maxRecordsCount, maxRecords, getRecordsRetrievalStrategy, diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java index 731c4653..0636baea 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/BlockingGetRecordsCacheTest.java @@ -40,7 +40,6 @@ import com.amazonaws.services.kinesis.model.Record; @RunWith(MockitoJUnitRunner.class) public class BlockingGetRecordsCacheTest { private static final int MAX_RECORDS_PER_COUNT = 10_000; - private static final long IDLE_MILLIS_BETWEEN_CALLS = 500L; @Mock private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; @@ -53,7 +52,7 @@ public class BlockingGetRecordsCacheTest { @Before public void setup() { records = new ArrayList<>(); - blockingGetRecordsCache = new BlockingGetRecordsCache(MAX_RECORDS_PER_COUNT, getRecordsRetrievalStrategy, IDLE_MILLIS_BETWEEN_CALLS); + blockingGetRecordsCache = new BlockingGetRecordsCache(MAX_RECORDS_PER_COUNT, getRecordsRetrievalStrategy); when(getRecordsRetrievalStrategy.getRecords(eq(MAX_RECORDS_PER_COUNT))).thenReturn(getRecordsResult); when(getRecordsResult.getRecords()).thenReturn(records); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 0bd2f31a..efb9d43c 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -339,8 +339,7 @@ public class ShardConsumerTest { KinesisDataFetcher dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, - new SynchronousGetRecordsRetrievalStrategy(dataFetcher), - 0L)); + new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); ShardConsumer consumer = @@ -469,8 +468,7 @@ public class ShardConsumerTest { KinesisDataFetcher dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, - new SynchronousGetRecordsRetrievalStrategy(dataFetcher), - 0L)); + new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); ShardConsumer consumer = From 821b0cbd0fc315d9eca192454ab0b42af5d89ae6 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Tue, 24 Oct 2017 09:38:49 -0700 Subject: [PATCH 20/35] Release Notes for 1.8.7 of the Amazon Kinesis Client Library for Java (#257) Don't add a delay for synchronous requests to Kinesis Removes a delay that had been added for synchronous GetRecords calls to Kinesis --- META-INF/MANIFEST.MF | 2 +- README.md | 5 +++++ pom.xml | 2 +- .../lib/worker/KinesisClientLibConfiguration.java | 2 +- 4 files changed, 8 insertions(+), 3 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index c48417b0..3c6411d8 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.6 +Bundle-Version: 1.8.7 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.8 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/README.md b/README.md index ee1b7f78..c612845f 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,11 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.7 +* Don't add a delay for synchronous requests to Kinesis + Removes a delay that had been added for synchronous `GetRecords` calls to Kinesis. + * [PR #256](https://github.com/awslabs/amazon-kinesis-client/pull/256) + ### Release 1.8.6 * Add prefetching of records from Kinesis Prefetching will retrieve and queue additional records from Kinesis while the application is processing existing records. diff --git a/pom.xml b/pom.xml index 46bba4ca..88a1a79a 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.6 + 1.8.7 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index da2610be..a37daa42 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.6"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.7"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From cbcc898d71e257ca02296e3e2521ac63c4847fd7 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Wed, 25 Oct 2017 08:09:44 -0700 Subject: [PATCH 21/35] Updating version to 1.8.8-SNAPSHOT. (#258) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 88a1a79a..b07d9ac0 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.7 + 1.8.8-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 7032ea67eced2745bba8090405da9fc50859dad7 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Wed, 25 Oct 2017 08:11:20 -0700 Subject: [PATCH 22/35] Spurious update fix (#247) * Handle spurious lease renewal failures gracefully. If the request to conditionally update a lease counter in DynamoDB fails, it's considered a failure to renew the lease. This is a good thing, except if the request failure was just because of connectivity problems. In this case the counter *did* update in DynamoDB, but the Dynamo client retries the request which then fails the update condition (since the lease counter no longer matches expected value). To handle this gracefully we opt to get the lease record from Dynamo and examine the lease owner and counter. If it matches what we were expecting, then we consider renewal a success. --- .../kinesis/leases/impl/LeaseManager.java | 15 +++++- .../leases/impl/LeaseIntegrationTest.java | 19 +++---- .../impl/LeaseManagerIntegrationTest.java | 21 ++++---- .../impl/LeaseRenewerIntegrationTest.java | 49 ++++++++++--------- .../leases/impl/TestHarnessBuilder.java | 33 ++++++++----- 5 files changed, 82 insertions(+), 55 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java index a2bf33a2..9dc2a4a3 100644 --- a/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/leases/impl/LeaseManager.java @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import com.amazonaws.services.kinesis.leases.util.DynamoUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -386,7 +387,19 @@ public class LeaseManager implements ILeaseManager { + " because the lease counter was not " + lease.getLeaseCounter()); } - return false; + // If we had a spurious retry during the Dynamo update, then this conditional PUT failure + // might be incorrect. So, we get the item straight away and check if the lease owner + lease counter + // are what we expected. + String expectedOwner = lease.getLeaseOwner(); + Long expectedCounter = lease.getLeaseCounter() + 1; + T updatedLease = getLease(lease.getLeaseKey()); + if (updatedLease == null || !expectedOwner.equals(updatedLease.getLeaseOwner()) || + !expectedCounter.equals(updatedLease.getLeaseCounter())) { + return false; + } + + LOG.info("Detected spurious renewal failure for lease with key " + lease.getLeaseKey() + + ", but recovered"); } catch (AmazonClientException e) { throw convertAndRethrowExceptions("renew", lease.getLeaseKey(), e); } diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseIntegrationTest.java index 57a9c99b..e7ff0ebe 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseIntegrationTest.java @@ -1,21 +1,22 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; import java.util.logging.Logger; +import com.amazonaws.services.kinesis.leases.exceptions.LeasingException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.Ignore; diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java index 23cc9fc1..dcaedc38 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseManagerIntegrationTest.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; @@ -108,7 +108,8 @@ public class LeaseManagerIntegrationTest extends LeaseIntegrationTest { KinesisClientLease leaseCopy = leaseManager.getLease(lease.getLeaseKey()); - leaseManager.renewLease(lease); + // lose lease + leaseManager.takeLease(lease, "bar"); Assert.assertFalse(leaseManager.renewLease(leaseCopy)); } diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseRenewerIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseRenewerIntegrationTest.java index 9792d006..8ad19d34 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseRenewerIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/LeaseRenewerIntegrationTest.java @@ -1,30 +1,29 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.Executors; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.leases.exceptions.LeasingException; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseRenewer; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.Executors; public class LeaseRenewerIntegrationTest extends LeaseIntegrationTest { @@ -58,7 +57,9 @@ public class LeaseRenewerIntegrationTest extends LeaseIntegrationTest { builder.addLeasesToRenew(renewer, "1", "2"); KinesisClientLease renewedLease = builder.renewMutateAssert(renewer, "1", "2").get("2"); - leaseManager.updateLease(renewedLease); + // lose lease 2 + leaseManager.takeLease(renewedLease, "bar"); + builder.renewMutateAssert(renewer, "1"); } @@ -96,9 +97,9 @@ public class LeaseRenewerIntegrationTest extends LeaseIntegrationTest { public void testGetCurrentlyHeldLeases() throws LeasingException { TestHarnessBuilder builder = new TestHarnessBuilder(leaseManager); - KinesisClientLease lease2 = builder.withLease("1", "foo").withLease("2", "foo").build().get("2"); + builder.withLease("1", "foo").withLease("2", "foo").build(); builder.addLeasesToRenew(renewer, "1", "2"); - builder.renewMutateAssert(renewer, "1", "2"); + KinesisClientLease lease2 = builder.renewMutateAssert(renewer, "1", "2").get("2"); // This should be a copy that doesn't get updated Map heldLeases = renewer.getCurrentlyHeldLeases(); @@ -106,7 +107,9 @@ public class LeaseRenewerIntegrationTest extends LeaseIntegrationTest { Assert.assertEquals((Long) 1L, heldLeases.get("1").getLeaseCounter()); Assert.assertEquals((Long) 1L, heldLeases.get("2").getLeaseCounter()); - leaseManager.updateLease(lease2); // lose lease 2 + // lose lease 2 + leaseManager.takeLease(lease2, "bar"); + // Do another renewal and make sure the copy doesn't change builder.renewMutateAssert(renewer, "1"); @@ -176,7 +179,7 @@ public class LeaseRenewerIntegrationTest extends LeaseIntegrationTest { KinesisClientLease lease = renewer.getCurrentlyHeldLease("1"); // cause lease loss such that the renewer knows the lease has been lost when update is called - leaseManager.renewLease(lease); + leaseManager.takeLease(lease, "bar"); builder.renewMutateAssert(renewer); lease.setCheckpoint(new ExtendedSequenceNumber("new checkpoint")); @@ -195,7 +198,7 @@ public class LeaseRenewerIntegrationTest extends LeaseIntegrationTest { KinesisClientLease lease = renewer.getCurrentlyHeldLease("1"); // cause lease loss such that the renewer knows the lease has been lost when update is called - leaseManager.renewLease(lease); + leaseManager.takeLease(lease, "bar"); builder.renewMutateAssert(renewer); // regain the lease diff --git a/src/test/java/com/amazonaws/services/kinesis/leases/impl/TestHarnessBuilder.java b/src/test/java/com/amazonaws/services/kinesis/leases/impl/TestHarnessBuilder.java index 6b6d673c..0dfbb568 100644 --- a/src/test/java/com/amazonaws/services/kinesis/leases/impl/TestHarnessBuilder.java +++ b/src/test/java/com/amazonaws/services/kinesis/leases/impl/TestHarnessBuilder.java @@ -1,16 +1,16 @@ /* - * Copyright 2012-2016 Amazon.com, Inc. or its affiliates. All Rights Reserved. + * Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved. * - * Licensed under the Amazon Software License (the "License"). - * You may not use this file except in compliance with the License. - * A copy of the License is located at + * Licensed under the Amazon Software License (the "License"). + * You may not use this file except in compliance with the License. + * A copy of the License is located at * - * http://aws.amazon.com/asl/ + * http://aws.amazon.com/asl/ * - * or in the "license" file accompanying this file. This file is distributed - * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either - * express or implied. See the License for the specific language governing - * permissions and limitations under the License. + * or in the "license" file accompanying this file. This file is distributed + * on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing + * permissions and limitations under the License. */ package com.amazonaws.services.kinesis.leases.impl; @@ -35,6 +35,7 @@ public class TestHarnessBuilder { private Map leases = new HashMap(); private KinesisClientLeaseManager leaseManager; + private Map originalLeases = new HashMap<>(); private Callable timeProvider = new Callable() { @@ -54,6 +55,15 @@ public class TestHarnessBuilder { } public TestHarnessBuilder withLease(String shardId, String owner) { + KinesisClientLease lease = createLease(shardId, owner); + KinesisClientLease originalLease = createLease(shardId, owner); + + leases.put(shardId, lease); + originalLeases.put(shardId, originalLease); + return this; + } + + private KinesisClientLease createLease(String shardId, String owner) { KinesisClientLease lease = new KinesisClientLease(); lease.setCheckpoint(new ExtendedSequenceNumber("checkpoint")); lease.setOwnerSwitchesSinceCheckpoint(0L); @@ -62,8 +72,7 @@ public class TestHarnessBuilder { lease.setParentShardIds(Collections.singleton("parentShardId")); lease.setLeaseKey(shardId); - leases.put(shardId, lease); - return this; + return lease; } public Map build() throws LeasingException { @@ -147,7 +156,7 @@ public class TestHarnessBuilder { Assert.assertEquals(renewedShardIds.length, heldLeases.size()); for (String shardId : renewedShardIds) { - KinesisClientLease original = leases.get(shardId); + KinesisClientLease original = originalLeases.get(shardId); Assert.assertNotNull(original); KinesisClientLease actual = heldLeases.get(shardId); From 3de901ea9327370ed732af86c4d4999c8d99541c Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Wed, 25 Oct 2017 15:38:45 -0700 Subject: [PATCH 23/35] Adding logging for long running tasks. (#259) Allow enabling logging for long running tasks. --- .../worker/KinesisClientLibConfiguration.java | 14 ++++++ .../lib/worker/ShardConsumer.java | 12 +++-- .../lib/worker/ShardConsumerTest.java | 49 +++++++++++++++++++ 3 files changed, 72 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index a37daa42..fae780f5 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -234,6 +234,9 @@ public class KinesisClientLibConfiguration { @Getter private RecordsFetcherFactory recordsFetcherFactory; + + @Getter + private Optional logWarningForTaskAfterMillis = Optional.empty(); /** * Constructor. @@ -1355,4 +1358,15 @@ public class KinesisClientLibConfiguration { this.recordsFetcherFactory.setIdleMillisBetweenCalls(idleMillisBetweenCalls); return this; } + + /** + * @param logWarningForTaskAfterMillis Logs warn message if as task is held in a task for more than the set + * time. + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withLogWarningForTaskAfterMillis(long logWarningForTaskAfterMillis) { + checkIsValuePositive("LogProcessTaskStatusAfterInMillis", logWarningForTaskAfterMillis); + this.logWarningForTaskAfterMillis = Optional.of(logWarningForTaskAfterMillis); + return this; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index d7a5545e..14a1d08c 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -283,11 +283,17 @@ class ShardConsumer { } } } else { + final long timeElapsed = System.currentTimeMillis() - currentTaskSubmitTime; + final String commonMessage = String.format("Previous %s task still pending for shard %s since %d ms ago. ", + currentTask.getTaskType(), shardInfo.getShardId(), timeElapsed); if (LOG.isDebugEnabled()) { - LOG.debug("Previous " + currentTask.getTaskType() + " task still pending for shard " - + shardInfo.getShardId() + " since " + (System.currentTimeMillis() - currentTaskSubmitTime) - + " ms ago" + ". Not submitting new task."); + LOG.debug(commonMessage + "Not submitting new task."); } + config.getLogWarningForTaskAfterMillis().ifPresent(value -> { + if (timeElapsed > value) { + LOG.warn(commonMessage); + } + }); } return submittedNewTask; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index efb9d43c..9a7f2234 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -27,6 +27,7 @@ import static org.mockito.Matchers.argThat; import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -43,6 +44,7 @@ import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; @@ -119,6 +121,7 @@ public class ShardConsumerTest { recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory(maxRecords)); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); + when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.empty()); } /** @@ -645,6 +648,52 @@ public class ShardConsumerTest { assertEquals(shardConsumer.getGetRecordsCache().getGetRecordsRetrievalStrategy().getClass(), AsynchronousGetRecordsRetrievalStrategy.class); } + + @SuppressWarnings("unchecked") + @Test + public void testLongRunningTasks() throws InterruptedException { + final long sleepTime = 1000L; + ExecutorService mockExecutorService = mock(ExecutorService.class); + Future mockFuture = mock(Future.class); + + when(mockExecutorService.submit(any(ITask.class))).thenReturn(mockFuture); + when(mockFuture.isDone()).thenReturn(false); + when(mockFuture.isCancelled()).thenReturn(false); + when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.of(sleepTime)); + + ShardInfo shardInfo = new ShardInfo("s-0-0", "testToken", null, ExtendedSequenceNumber.LATEST); + StreamConfig streamConfig = new StreamConfig( + streamProxy, + 1, + 10, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, + INITIAL_POSITION_LATEST); + + ShardConsumer shardConsumer = new ShardConsumer( + shardInfo, + streamConfig, + checkpoint, + processor, + null, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + mockExecutorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + config); + + shardConsumer.consumeShard(); + + Thread.sleep(sleepTime); + + shardConsumer.consumeShard(); + + verify(config).getLogWarningForTaskAfterMillis(); + verify(mockFuture).isDone(); + verify(mockFuture).isCancelled(); + } //@formatter:off (gets the formatting wrong) private void verifyConsumedRecords(List expectedRecords, From 5c3ff2b31edd65e578095ff2adb5200b0efc6a45 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Wed, 8 Nov 2017 12:03:09 -0800 Subject: [PATCH 24/35] Handle Expired Iterators Correctly Fix for the lease losses in the PrefetchCache and AsyncGetRecordsStrategy caused due to ExpiredIteratorException. (#263) --- ...ynchronousGetRecordsRetrievalStrategy.java | 57 +++++++++++-------- .../worker/GetRecordsRetrievalStrategy.java | 7 +++ .../lib/worker/KinesisDataFetcher.java | 21 +++++++ .../lib/worker/PrefetchGetRecordsCache.java | 21 ++++++- .../lib/worker/RecordsFetcherFactory.java | 3 +- .../worker/SimpleRecordsFetcherFactory.java | 7 ++- ...ynchronousGetRecordsRetrievalStrategy.java | 5 ++ ...cordsRetrievalStrategyIntegrationTest.java | 23 ++++++++ ...ronousGetRecordsRetrievalStrategyTest.java | 27 +++++++++ .../lib/worker/KinesisDataFetcherTest.java | 40 +++++++++++++ ...refetchGetRecordsCacheIntegrationTest.java | 39 ++++++++++--- .../worker/PrefetchGetRecordsCacheTest.java | 25 +++++++- .../lib/worker/RecordsFetcherFactoryTest.java | 5 +- .../lib/worker/ShardConsumerTest.java | 8 ++- .../clientlibrary/lib/worker/WorkerTest.java | 4 +- 15 files changed, 248 insertions(+), 44 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java index 2e3cbd9e..2db74fba5 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java @@ -16,7 +16,6 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; @@ -31,6 +30,7 @@ import java.util.function.Supplier; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingScope; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -81,33 +81,39 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie CompletionService completionService = completionServiceSupplier.get(); Set> futures = new HashSet<>(); Callable retrieverCall = createRetrieverCallable(maxRecords); - while (true) { - try { - futures.add(completionService.submit(retrieverCall)); - } catch (RejectedExecutionException e) { - log.warn("Out of resources, unable to start additional requests."); - } + try { + while (true) { + try { + futures.add(completionService.submit(retrieverCall)); + } catch (RejectedExecutionException e) { + log.warn("Out of resources, unable to start additional requests."); + } - try { - Future resultFuture = completionService.poll(retryGetRecordsInSeconds, - TimeUnit.SECONDS); - if (resultFuture != null) { - // - // Fix to ensure that we only let the shard iterator advance when we intend to return the result - // to the caller. This ensures that the shard iterator is consistently advance in step with - // what the caller sees. - // - result = resultFuture.get().accept(); + try { + Future resultFuture = completionService.poll(retryGetRecordsInSeconds, + TimeUnit.SECONDS); + if (resultFuture != null) { + // + // Fix to ensure that we only let the shard iterator advance when we intend to return the result + // to the caller. This ensures that the shard iterator is consistently advance in step with + // what the caller sees. + // + result = resultFuture.get().accept(); + break; + } + } catch (ExecutionException e) { + if (e.getCause() instanceof ExpiredIteratorException) { + throw (ExpiredIteratorException) e.getCause(); + } + log.error("ExecutionException thrown while trying to get records", e); + } catch (InterruptedException e) { + log.error("Thread was interrupted", e); break; } - } catch (ExecutionException e) { - log.error("ExecutionException thrown while trying to get records", e); - } catch (InterruptedException e) { - log.error("Thread was interrupted", e); - break; } + } finally { + futures.forEach(f -> f.cancel(true)); } - futures.forEach(f -> f.cancel(true)); return result; } @@ -140,4 +146,9 @@ public class AsynchronousGetRecordsRetrievalStrategy implements GetRecordsRetrie new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadNameFormat).build(), new ThreadPoolExecutor.AbortPolicy()); } + + @Override + public KinesisDataFetcher getDataFetcher() { + return dataFetcher; + } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java index 8f7afe25..4f474887 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/GetRecordsRetrievalStrategy.java @@ -44,4 +44,11 @@ public interface GetRecordsRetrievalStrategy { * @return true if the strategy has been shutdown, false otherwise. */ boolean isShutdown(); + + /** + * Returns the KinesisDataFetcher used to getRecords from Kinesis. + * + * @return KinesisDataFetcher + */ + KinesisDataFetcher getDataFetcher(); } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java index c2ba9d15..0bd4bee3 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcher.java @@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import java.util.Collections; import java.util.Date; +import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -27,6 +28,8 @@ import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.util.CollectionUtils; +import com.google.common.collect.Iterables; import lombok.Data; @@ -42,6 +45,8 @@ class KinesisDataFetcher { private final String shardId; private boolean isShardEndReached; private boolean isInitialized; + private String lastKnownSequenceNumber; + private InitialPositionInStreamExtended initialPositionInStream; /** * @@ -108,6 +113,9 @@ class KinesisDataFetcher { @Override public GetRecordsResult accept() { nextIterator = result.getNextShardIterator(); + if (!CollectionUtils.isNullOrEmpty(result.getRecords())) { + lastKnownSequenceNumber = Iterables.getLast(result.getRecords()).getSequenceNumber(); + } if (nextIterator == null) { isShardEndReached = true; } @@ -161,6 +169,8 @@ class KinesisDataFetcher { if (nextIterator == null) { isShardEndReached = true; } + this.lastKnownSequenceNumber = sequenceNumber; + this.initialPositionInStream = initialPositionInStream; } /** @@ -217,6 +227,17 @@ class KinesisDataFetcher { return iterator; } + /** + * Gets a new iterator from the last known sequence number i.e. the sequence number of the last record from the last + * getRecords call. + */ + public void restartIterator() { + if (StringUtils.isEmpty(lastKnownSequenceNumber) || initialPositionInStream == null) { + throw new IllegalStateException("Make sure to initialize the KinesisDataFetcher before restarting the iterator."); + } + advanceIteratorTo(lastKnownSequenceNumber, initialPositionInStream); + } + /** * @return the shardEndReached */ diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java index 06e77c8c..982d70cc 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCache.java @@ -23,10 +23,13 @@ import java.util.concurrent.LinkedBlockingQueue; import org.apache.commons.lang.Validate; import com.amazonaws.SdkClientException; +import com.amazonaws.services.cloudwatch.model.StandardUnit; import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingFactory; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; +import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsResult; import lombok.NonNull; @@ -42,6 +45,7 @@ import lombok.extern.apachecommons.CommonsLog; */ @CommonsLog public class PrefetchGetRecordsCache implements GetRecordsCache { + private static final String EXPIRED_ITERATOR_METRIC = "ExpiredIterator"; LinkedBlockingQueue getRecordsResultQueue; private int maxPendingProcessRecordsInput; private int maxByteSize; @@ -56,6 +60,8 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { private PrefetchCounters prefetchCounters; private boolean started = false; private final String operation; + private final KinesisDataFetcher dataFetcher; + private final String shardId; /** * Constructor for the PrefetchGetRecordsCache. This cache prefetches records from Kinesis and stores them in a @@ -76,9 +82,10 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { final int maxRecordsPerCall, @NonNull final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, @NonNull final ExecutorService executorService, - long idleMillisBetweenCalls, + final long idleMillisBetweenCalls, @NonNull final IMetricsFactory metricsFactory, - @NonNull String operation) { + @NonNull final String operation, + @NonNull final String shardId) { this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; this.maxRecordsPerCall = maxRecordsPerCall; this.maxPendingProcessRecordsInput = maxPendingProcessRecordsInput; @@ -92,6 +99,8 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon(); Validate.notEmpty(operation, "Operation cannot be empty"); this.operation = operation; + this.dataFetcher = this.getRecordsRetrievalStrategy.getDataFetcher(); + this.shardId = shardId; } @Override @@ -162,6 +171,14 @@ public class PrefetchGetRecordsCache implements GetRecordsCache { prefetchCounters.added(processRecordsInput); } catch (InterruptedException e) { log.info("Thread was interrupted, indicating shutdown was called on the cache."); + } catch (ExpiredIteratorException e) { + log.info(String.format("ShardId %s: getRecords threw ExpiredIteratorException - restarting" + + " after greatest seqNum passed to customer", shardId), e); + + MetricsHelper.getMetricsScope().addData(EXPIRED_ITERATOR_METRIC, 1, StandardUnit.Count, + MetricsLevel.SUMMARY); + + dataFetcher.restartIterator(); } catch (SdkClientException e) { log.error("Exception thrown while fetching records from Kinesis", e); } catch (Throwable e) { diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java index be8316d7..afc6c4f2 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java @@ -29,7 +29,8 @@ public interface RecordsFetcherFactory { * * @return GetRecordsCache used to get records from Kinesis. */ - GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory); + GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, + IMetricsFactory metricsFactory); /** * Sets the maximum number of ProcessRecordsInput objects the GetRecordsCache can hold, before further requests are diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java index 44c93e7b..bd33fd98 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java @@ -18,6 +18,7 @@ import java.util.concurrent.Executors; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.google.common.util.concurrent.ThreadFactoryBuilder; + import lombok.extern.apachecommons.CommonsLog; @CommonsLog @@ -34,7 +35,8 @@ public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory { } @Override - public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory) { + public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, + IMetricsFactory metricsFactory) { if(dataFetchingStrategy.equals(DataFetchingStrategy.DEFAULT)) { return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy); } else { @@ -46,7 +48,8 @@ public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory { .build()), idleMillisBetweenCalls, metricsFactory, - "ProcessTask"); + "ProcessTask", + shardId); } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java index c862c348..f4209189 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SynchronousGetRecordsRetrievalStrategy.java @@ -42,4 +42,9 @@ public class SynchronousGetRecordsRetrievalStrategy implements GetRecordsRetriev public boolean isShutdown() { return false; } + + @Override + public KinesisDataFetcher getDataFetcher() { + return dataFetcher; + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java index 30b877e8..37f58c1c 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyIntegrationTest.java @@ -38,17 +38,21 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; import org.mockito.runners.MockitoJUnitRunner; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.mockito.stubbing.Answer; @RunWith(MockitoJUnitRunner.class) public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { @@ -133,6 +137,24 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { verify(mockFuture).get(); assertNull(getRecordsResult); } + + @Test (expected = ExpiredIteratorException.class) + public void testExpiredIteratorExcpetion() throws InterruptedException { + when(dataFetcher.getRecords(eq(numberOfRecords))).thenAnswer(new Answer() { + @Override + public DataFetcherResult answer(final InvocationOnMock invocationOnMock) throws Throwable { + Thread.sleep(SLEEP_GET_RECORDS_IN_SECONDS * 1000); + throw new ExpiredIteratorException("ExpiredIterator"); + } + }); + + try { + getRecordsRetrivalStrategy.getRecords(numberOfRecords); + } finally { + verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(eq(numberOfRecords)); + verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any()); + } + } private int getLeastNumberOfCalls() { int leastNumberOfCalls = 0; @@ -163,6 +185,7 @@ public class AsynchronousGetRecordsRetrievalStrategyIntegrationTest { } catch (InterruptedException e) { // Do nothing } + return result; } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java index aa9e9a24..151300de 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategyTest.java @@ -20,20 +20,25 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; @@ -153,5 +158,27 @@ public class AsynchronousGetRecordsRetrievalStrategyTest { assertThat(actualResult, equalTo(expectedResults)); } + + @Test (expected = ExpiredIteratorException.class) + public void testExpiredIteratorExceptionCase() throws Exception { + AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, + executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID); + Future successfulFuture2 = mock(Future.class); + + when(executorService.isShutdown()).thenReturn(false); + when(completionService.submit(any())).thenReturn(successfulFuture, successfulFuture2); + when(completionService.poll(anyLong(), any())).thenReturn(null).thenReturn(successfulFuture); + when(successfulFuture.get()).thenThrow(new ExecutionException(new ExpiredIteratorException("ExpiredException"))); + + try { + strategy.getRecords(10); + } finally { + verify(executorService).isShutdown(); + verify(completionService, times(2)).submit(any()); + verify(completionService, times(2)).poll(eq(RETRY_GET_RECORDS_IN_SECONDS), eq(TimeUnit.SECONDS)); + verify(successfulFuture).cancel(eq(true)); + verify(successfulFuture2).cancel(eq(true)); + } + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java index 6648b919..fbe720ae 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisDataFetcherTest.java @@ -32,6 +32,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import java.util.ArrayList; +import java.util.Collections; import java.util.Date; import java.util.List; @@ -273,6 +274,45 @@ public class KinesisDataFetcherTest { verify(kinesisProxy, never()).get(anyString(), anyInt()); } + + @Test + public void testRestartIterator() { + GetRecordsResult getRecordsResult = mock(GetRecordsResult.class); + GetRecordsResult restartGetRecordsResult = new GetRecordsResult(); + Record record = mock(Record.class); + final String initialIterator = "InitialIterator"; + final String nextShardIterator = "NextShardIterator"; + final String restartShardIterator = "RestartIterator"; + final String sequenceNumber = "SequenceNumber"; + final String iteratorType = "AT_SEQUENCE_NUMBER"; + KinesisProxy kinesisProxy = mock(KinesisProxy.class); + KinesisDataFetcher fetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); + + when(kinesisProxy.getIterator(eq(SHARD_ID), eq(InitialPositionInStream.LATEST.toString()))).thenReturn(initialIterator); + when(kinesisProxy.get(eq(initialIterator), eq(10))).thenReturn(getRecordsResult); + when(getRecordsResult.getRecords()).thenReturn(Collections.singletonList(record)); + when(getRecordsResult.getNextShardIterator()).thenReturn(nextShardIterator); + when(record.getSequenceNumber()).thenReturn(sequenceNumber); + + fetcher.initialize(InitialPositionInStream.LATEST.toString(), INITIAL_POSITION_LATEST); + verify(kinesisProxy).getIterator(eq(SHARD_ID), eq(InitialPositionInStream.LATEST.toString())); + Assert.assertEquals(getRecordsResult, fetcher.getRecords(10).accept()); + verify(kinesisProxy).get(eq(initialIterator), eq(10)); + + when(kinesisProxy.getIterator(eq(SHARD_ID), eq(iteratorType), eq(sequenceNumber))).thenReturn(restartShardIterator); + when(kinesisProxy.get(eq(restartShardIterator), eq(10))).thenReturn(restartGetRecordsResult); + + fetcher.restartIterator(); + Assert.assertEquals(restartGetRecordsResult, fetcher.getRecords(10).accept()); + verify(kinesisProxy).getIterator(eq(SHARD_ID), eq(iteratorType), eq(sequenceNumber)); + verify(kinesisProxy).get(eq(restartShardIterator), eq(10)); + } + + @Test (expected = IllegalStateException.class) + public void testRestartIteratorNotInitialized() { + KinesisDataFetcher dataFetcher = new KinesisDataFetcher(kinesisProxy, SHARD_INFO); + dataFetcher.restartIterator(); + } private DataFetcherResult assertAdvanced(KinesisDataFetcher dataFetcher, GetRecordsResult expectedResult, String previousValue, String nextValue) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java index 37d0e446..e24d5bb0 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheIntegrationTest.java @@ -20,6 +20,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; @@ -31,19 +33,19 @@ import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; -import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; -import com.amazonaws.services.kinesis.model.Record; - import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.Record; @@ -70,14 +72,13 @@ public class PrefetchGetRecordsCacheIntegrationTest { @Mock private IKinesisProxy proxy; - @Mock private ShardInfo shardInfo; @Before public void setup() { records = new ArrayList<>(); - dataFetcher = new KinesisDataFetcherForTest(proxy, shardInfo); + dataFetcher = spy(new KinesisDataFetcherForTest(proxy, shardInfo)); getRecordsRetrievalStrategy = spy(new SynchronousGetRecordsRetrievalStrategy(dataFetcher)); executorService = spy(Executors.newFixedThreadPool(1)); @@ -89,7 +90,8 @@ public class PrefetchGetRecordsCacheIntegrationTest { executorService, IDLE_MILLIS_BETWEEN_CALLS, new NullMetricsFactory(), - operation); + operation, + "test-shard"); } @Test @@ -135,7 +137,8 @@ public class PrefetchGetRecordsCacheIntegrationTest { executorService2, IDLE_MILLIS_BETWEEN_CALLS, new NullMetricsFactory(), - operation); + operation, + "test-shard-2"); getRecordsCache.start(); sleep(IDLE_MILLIS_BETWEEN_CALLS); @@ -167,6 +170,26 @@ public class PrefetchGetRecordsCacheIntegrationTest { verify(getRecordsRetrievalStrategy2).shutdown(); } + @Test + public void testExpiredIteratorException() { + when(dataFetcher.getRecords(eq(MAX_RECORDS_PER_CALL))).thenAnswer(new Answer() { + @Override + public DataFetcherResult answer(final InvocationOnMock invocationOnMock) throws Throwable { + throw new ExpiredIteratorException("ExpiredIterator"); + } + }).thenCallRealMethod(); + doNothing().when(dataFetcher).restartIterator(); + + getRecordsCache.start(); + sleep(IDLE_MILLIS_BETWEEN_CALLS); + + ProcessRecordsInput processRecordsInput = getRecordsCache.getNextResult(); + + assertNotNull(processRecordsInput); + assertTrue(processRecordsInput.getRecords().isEmpty()); + verify(dataFetcher).restartIterator(); + } + @After public void shutdown() { getRecordsCache.shutdown(); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java index 6091baa9..2b650866 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/PrefetchGetRecordsCacheTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -36,7 +37,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; import java.util.stream.IntStream; -import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -45,6 +45,8 @@ import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; import com.amazonaws.services.kinesis.clientlibrary.types.ProcessRecordsInput; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.Record; @@ -66,6 +68,8 @@ public class PrefetchGetRecordsCacheTest { private GetRecordsResult getRecordsResult; @Mock private Record record; + @Mock + private KinesisDataFetcher dataFetcher; private List records; private ExecutorService executorService; @@ -75,6 +79,8 @@ public class PrefetchGetRecordsCacheTest { @Before public void setup() { + when(getRecordsRetrievalStrategy.getDataFetcher()).thenReturn(dataFetcher); + executorService = spy(Executors.newFixedThreadPool(1)); getRecordsCache = new PrefetchGetRecordsCache( MAX_SIZE, @@ -85,7 +91,8 @@ public class PrefetchGetRecordsCacheTest { executorService, IDLE_MILLIS_BETWEEN_CALLS, new NullMetricsFactory(), - operation); + operation, + "shardId"); spyQueue = spy(getRecordsCache.getRecordsResultQueue); records = spy(new ArrayList<>()); @@ -194,6 +201,20 @@ public class PrefetchGetRecordsCacheTest { when(executorService.isShutdown()).thenReturn(true); getRecordsCache.getNextResult(); } + + @Test + public void testExpiredIteratorException() { + getRecordsCache.start(); + + when(getRecordsRetrievalStrategy.getRecords(MAX_RECORDS_PER_CALL)).thenThrow(ExpiredIteratorException.class).thenReturn(getRecordsResult); + doNothing().when(dataFetcher).restartIterator(); + + getRecordsCache.getNextResult(); + + sleep(1000); + + verify(dataFetcher).restartIterator(); + } @After public void shutdown() { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java index 912804da..7107d0fd 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java @@ -1,7 +1,5 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; -import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; - import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; @@ -10,13 +8,14 @@ import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; + public class RecordsFetcherFactoryTest { private String shardId = "TestShard"; private RecordsFetcherFactory recordsFetcherFactory; @Mock private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; - @Mock private IMetricsFactory metricsFactory; diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 9a7f2234..f235ca93 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -343,7 +343,9 @@ public class ShardConsumerTest { getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); - when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); + when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), + any(IMetricsFactory.class))) + .thenReturn(getRecordsCache); ShardConsumer consumer = new ShardConsumer(shardInfo, @@ -472,7 +474,9 @@ public class ShardConsumerTest { getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); - when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); + when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), + any(IMetricsFactory.class))) + .thenReturn(getRecordsCache); ShardConsumer consumer = new ShardConsumer(shardInfo, diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index a8856a0b..fd3382a3 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -621,7 +621,9 @@ public class WorkerTest { RecordsFetcherFactory recordsFetcherFactory = mock(RecordsFetcherFactory.class); GetRecordsCache getRecordsCache = mock(GetRecordsCache.class); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); - when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); + when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), + any(IMetricsFactory.class))) + .thenReturn(getRecordsCache); when(getRecordsCache.getNextResult()).thenReturn(new ProcessRecordsInput().withRecords(Collections.emptyList()).withMillisBehindLatest(0L)); WorkerThread workerThread = runWorker(shardList, From 1abb41dbdbd074218427469b9ea1029285a544d0 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Fri, 10 Nov 2017 06:32:16 -0800 Subject: [PATCH 25/35] Correctly Send MaxRecords to SingleRecordsFetcherFactory Fixed #262 Fixes #262 Changing the signture of SingleRecordsFetcherFactory to no longer take maxRecords as the parameter to the constructor. Changed the createRecordsFetcher signature to take maxRecords as a parameter. (#264) --- .../lib/worker/KinesisClientLibConfiguration.java | 2 +- .../lib/worker/RecordsFetcherFactory.java | 3 ++- .../clientlibrary/lib/worker/ShardConsumer.java | 2 +- .../lib/worker/SimpleRecordsFetcherFactory.java | 9 ++------- .../lib/worker/RecordsFetcherFactoryTest.java | 6 +++--- .../clientlibrary/lib/worker/ShardConsumerTest.java | 10 +++++----- .../kinesis/clientlibrary/lib/worker/WorkerTest.java | 7 ++++--- 7 files changed, 18 insertions(+), 21 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index fae780f5..f24fc574 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -477,7 +477,7 @@ public class KinesisClientLibConfiguration { InitialPositionInStreamExtended.newInitialPosition(initialPositionInStream); this.skipShardSyncAtWorkerInitializationIfLeasesExist = DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST; this.shardPrioritization = DEFAULT_SHARD_PRIORITIZATION; - this.recordsFetcherFactory = new SimpleRecordsFetcherFactory(this.maxRecords); + this.recordsFetcherFactory = new SimpleRecordsFetcherFactory(); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java index afc6c4f2..c1a513a9 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactory.java @@ -26,11 +26,12 @@ public interface RecordsFetcherFactory { * @param getRecordsRetrievalStrategy GetRecordsRetrievalStrategy to be used with the GetRecordsCache * @param shardId ShardId of the shard that the fetcher will retrieve records for * @param metricsFactory MetricsFactory used to create metricScope + * @param maxRecords Max number of records to be returned in a single get call * * @return GetRecordsCache used to get records from Kinesis. */ GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, - IMetricsFactory metricsFactory); + IMetricsFactory metricsFactory, int maxRecords); /** * Sets the maximum number of ProcessRecordsInput objects the GetRecordsCache can hold, before further requests are diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 14a1d08c..d8aa88d1 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -235,7 +235,7 @@ class ShardConsumer { this.dataFetcher = kinesisDataFetcher; this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher( makeStrategy(this.dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, this.shardInfo), - this.getShardInfo().getShardId(), this.metricsFactory); + this.getShardInfo().getShardId(), this.metricsFactory, this.config.getMaxRecords()); } /** diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java index bd33fd98..79ad9f55 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/SimpleRecordsFetcherFactory.java @@ -23,20 +23,15 @@ import lombok.extern.apachecommons.CommonsLog; @CommonsLog public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory { - private final int maxRecords; private int maxPendingProcessRecordsInput = 3; private int maxByteSize = 8 * 1024 * 1024; private int maxRecordsCount = 30000; private long idleMillisBetweenCalls = 1500L; private DataFetchingStrategy dataFetchingStrategy = DataFetchingStrategy.DEFAULT; - - public SimpleRecordsFetcherFactory(int maxRecords) { - this.maxRecords = maxRecords; - } - + @Override public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, - IMetricsFactory metricsFactory) { + IMetricsFactory metricsFactory, int maxRecords) { if(dataFetchingStrategy.equals(DataFetchingStrategy.DEFAULT)) { return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy); } else { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java index 7107d0fd..d686c914 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordsFetcherFactoryTest.java @@ -22,13 +22,13 @@ public class RecordsFetcherFactoryTest { @Before public void setUp() { MockitoAnnotations.initMocks(this); - recordsFetcherFactory = new SimpleRecordsFetcherFactory(1); + recordsFetcherFactory = new SimpleRecordsFetcherFactory(); } @Test public void createDefaultRecordsFetcherTest() { GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId, - metricsFactory); + metricsFactory, 1); assertThat(recordsCache, instanceOf(BlockingGetRecordsCache.class)); } @@ -36,7 +36,7 @@ public class RecordsFetcherFactoryTest { public void createPrefetchRecordsFetcherTest() { recordsFetcherFactory.setDataFetchingStrategy(DataFetchingStrategy.PREFETCH_CACHED); GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId, - metricsFactory); + metricsFactory, 1); assertThat(recordsCache, instanceOf(PrefetchGetRecordsCache.class)); } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index f235ca93..516788c7 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.argThat; import static org.mockito.Mockito.atLeastOnce; @@ -97,7 +98,6 @@ public class ShardConsumerTest { // Use Executors.newFixedThreadPool since it returns ThreadPoolExecutor, which is // ... a non-final public class, and so can be mocked and spied. private final ExecutorService executorService = Executors.newFixedThreadPool(1); - private final int maxRecords = 500; private RecordsFetcherFactory recordsFetcherFactory; private GetRecordsCache getRecordsCache; @@ -119,7 +119,7 @@ public class ShardConsumerTest { public void setup() { getRecordsCache = null; - recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory(maxRecords)); + recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); when(config.getLogWarningForTaskAfterMillis()).thenReturn(Optional.empty()); } @@ -344,7 +344,7 @@ public class ShardConsumerTest { getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), - any(IMetricsFactory.class))) + any(IMetricsFactory.class), anyInt())) .thenReturn(getRecordsCache); ShardConsumer consumer = @@ -475,7 +475,7 @@ public class ShardConsumerTest { getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), - any(IMetricsFactory.class))) + any(IMetricsFactory.class), anyInt())) .thenReturn(getRecordsCache); ShardConsumer consumer = @@ -571,7 +571,7 @@ public class ShardConsumerTest { final ExtendedSequenceNumber checkpointSequenceNumber = new ExtendedSequenceNumber("123"); final ExtendedSequenceNumber pendingCheckpointSequenceNumber = new ExtendedSequenceNumber("999"); when(leaseManager.getLease(anyString())).thenReturn(null); - when(config.getRecordsFetcherFactory()).thenReturn(new SimpleRecordsFetcherFactory(2)); + when(config.getRecordsFetcherFactory()).thenReturn(new SimpleRecordsFetcherFactory()); when(checkpoint.getCheckpointObject(anyString())).thenReturn( new Checkpoint(checkpointSequenceNumber, pendingCheckpointSequenceNumber)); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index fd3382a3..ce406dce 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.eq; @@ -172,7 +173,7 @@ public class WorkerTest { @Before public void setup() { config = spy(new KinesisClientLibConfiguration("app", null, null, null)); - recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory(500)); + recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); } @@ -505,7 +506,7 @@ public class WorkerTest { lease.setCheckpoint(new ExtendedSequenceNumber("2")); initialLeases.add(lease); boolean callProcessRecordsForEmptyRecordList = true; - RecordsFetcherFactory recordsFetcherFactory = new SimpleRecordsFetcherFactory(500); + RecordsFetcherFactory recordsFetcherFactory = new SimpleRecordsFetcherFactory(); recordsFetcherFactory.setIdleMillisBetweenCalls(0L); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); runAndTestWorker(shardList, threadPoolSize, initialLeases, callProcessRecordsForEmptyRecordList, numberOfRecordsPerShard, config); @@ -622,7 +623,7 @@ public class WorkerTest { GetRecordsCache getRecordsCache = mock(GetRecordsCache.class); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), - any(IMetricsFactory.class))) + any(IMetricsFactory.class), anyInt())) .thenReturn(getRecordsCache); when(getRecordsCache.getNextResult()).thenReturn(new ProcessRecordsInput().withRecords(Collections.emptyList()).withMillisBehindLatest(0L)); From 09b312028a237f25f29c0ed6cd84c5ee56b30487 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Mon, 13 Nov 2017 09:45:40 -0800 Subject: [PATCH 26/35] Unit test fixes for retrying when requesting a shutdown. (#267) * Shutdown that throws an exception will be retried. Without this change a transient error on shutdown with reason terminate prevents child shards from starting. * Fixing the tests for the Shutdown fix. --- .../lib/worker/ShardConsumer.java | 2 +- .../lib/worker/ShardConsumerTest.java | 158 +++++++++++++++++- 2 files changed, 157 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index d8aa88d1..9fb8e8e9 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -412,7 +412,7 @@ class ShardConsumer { if (taskOutcome == TaskOutcome.END_OF_SHARD) { markForShutdown(ShutdownReason.TERMINATE); } - if (isShutdownRequested()) { + if (isShutdownRequested() && taskOutcome != TaskOutcome.FAILURE) { currentState = currentState.shutdownTransition(shutdownReason); } else if (taskOutcome == TaskOutcome.SUCCESSFUL) { if (currentState.getTaskType() == currentTask.getTaskType()) { diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 516788c7..8a91c6e6 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -42,6 +42,7 @@ import java.util.List; import java.util.ListIterator; import java.util.Objects; import java.util.Optional; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -69,12 +70,14 @@ import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProx import com.amazonaws.services.kinesis.clientlibrary.proxies.util.KinesisLocalFileDataCreator; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.InitializationInput; +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownInput; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; import com.amazonaws.services.kinesis.leases.impl.KinesisClientLease; import com.amazonaws.services.kinesis.leases.interfaces.ILeaseManager; import com.amazonaws.services.kinesis.metrics.impl.NullMetricsFactory; import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; /** @@ -102,6 +105,8 @@ public class ShardConsumerTest { private GetRecordsCache getRecordsCache; + private KinesisDataFetcher dataFetcher; + @Mock private IRecordProcessor processor; @Mock @@ -118,6 +123,7 @@ public class ShardConsumerTest { @Before public void setup() { getRecordsCache = null; + dataFetcher = null; recordsFetcherFactory = spy(new SimpleRecordsFetcherFactory()); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); @@ -339,7 +345,7 @@ public class ShardConsumerTest { ) ); - KinesisDataFetcher dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); + dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); @@ -422,6 +428,154 @@ public class ShardConsumerTest { file.delete(); } + private static final class TransientShutdownErrorTestStreamlet extends TestStreamlet { + private final CountDownLatch errorShutdownLatch = new CountDownLatch(1); + + @Override + public void shutdown(ShutdownInput input) { + ShutdownReason reason = input.getShutdownReason(); + if (reason.equals(ShutdownReason.TERMINATE) && errorShutdownLatch.getCount() > 0) { + errorShutdownLatch.countDown(); + throw new RuntimeException("test"); + } else { + super.shutdown(input); + } + } + } + + /** + * Test method for {@link ShardConsumer#consumeShard()} that ensures a transient error thrown from the record + * processor's shutdown method with reason terminate will be retried. + */ + @Test + public final void testConsumeShardWithTransientTerminateError() throws Exception { + int numRecs = 10; + BigInteger startSeqNum = BigInteger.ONE; + String streamShardId = "kinesis-0-0"; + String testConcurrencyToken = "testToken"; + List shardList = KinesisLocalFileDataCreator.createShardList(1, "kinesis-0-", startSeqNum); + // Close the shard so that shutdown is called with reason terminate + shardList.get(0).getSequenceNumberRange().setEndingSequenceNumber( + KinesisLocalFileProxy.MAX_SEQUENCE_NUMBER.subtract(BigInteger.ONE).toString()); + File file = KinesisLocalFileDataCreator.generateTempDataFile(shardList, numRecs, "unitTestSCT002"); + + IKinesisProxy fileBasedProxy = new KinesisLocalFileProxy(file.getAbsolutePath()); + + final int maxRecords = 2; + final int idleTimeMS = 0; // keep unit tests fast + ICheckpoint checkpoint = new InMemoryCheckpointImpl(startSeqNum.toString()); + checkpoint.setCheckpoint(streamShardId, ExtendedSequenceNumber.TRIM_HORIZON, testConcurrencyToken); + when(leaseManager.getLease(anyString())).thenReturn(null); + + TransientShutdownErrorTestStreamlet processor = new TransientShutdownErrorTestStreamlet(); + + StreamConfig streamConfig = + new StreamConfig(fileBasedProxy, + maxRecords, + idleTimeMS, + callProcessRecordsForEmptyRecordList, + skipCheckpointValidationValue, INITIAL_POSITION_LATEST); + + ShardInfo shardInfo = new ShardInfo(streamShardId, testConcurrencyToken, null, null); + + dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); + + getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, + new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); + when(recordsFetcherFactory.createRecordsFetcher(any(GetRecordsRetrievalStrategy.class), anyString(), + any(IMetricsFactory.class), anyInt())) + .thenReturn(getRecordsCache); + + RecordProcessorCheckpointer recordProcessorCheckpointer = new RecordProcessorCheckpointer( + shardInfo, + checkpoint, + new SequenceNumberValidator( + streamConfig.getStreamProxy(), + shardInfo.getShardId(), + streamConfig.shouldValidateSequenceNumberBeforeCheckpointing() + ) + ); + + ShardConsumer consumer = + new ShardConsumer(shardInfo, + streamConfig, + checkpoint, + processor, + recordProcessorCheckpointer, + leaseManager, + parentShardPollIntervalMillis, + cleanupLeasesOfCompletedShards, + executorService, + metricsFactory, + taskBackoffTimeMillis, + KinesisClientLibConfiguration.DEFAULT_SKIP_SHARD_SYNC_AT_STARTUP_IF_LEASES_EXIST, + dataFetcher, + Optional.empty(), + Optional.empty(), + config); + + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.WAITING_ON_PARENT_SHARDS))); + consumer.consumeShard(); // check on parent shards + Thread.sleep(50L); + consumer.consumeShard(); // start initialization + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.INITIALIZING))); + consumer.consumeShard(); // initialize + processor.getInitializeLatch().await(5, TimeUnit.SECONDS); + verify(getRecordsCache).start(); + + // We expect to process all records in numRecs calls + for (int i = 0; i < numRecs;) { + boolean newTaskSubmitted = consumer.consumeShard(); + if (newTaskSubmitted) { + LOG.debug("New processing task was submitted, call # " + i); + assertThat(consumer.getCurrentState(), is(equalTo(ConsumerStates.ShardConsumerState.PROCESSING))); + // CHECKSTYLE:IGNORE ModifiedControlVariable FOR NEXT 1 LINES + i += maxRecords; + } + Thread.sleep(50L); + } + + // Consume shards until shutdown terminate is called and it has thrown an exception + for (int i = 0; i < 100; i++) { + consumer.consumeShard(); + if (processor.errorShutdownLatch.await(50, TimeUnit.MILLISECONDS)) { + break; + } + } + assertEquals(0, processor.errorShutdownLatch.getCount()); + + // Wait for a retry of shutdown terminate that should succeed + for (int i = 0; i < 100; i++) { + consumer.consumeShard(); + if (processor.getShutdownLatch().await(50, TimeUnit.MILLISECONDS)) { + break; + } + } + assertEquals(0, processor.getShutdownLatch().getCount()); + + // Wait for shutdown complete now that terminate shutdown is successful + for (int i = 0; i < 100; i++) { + consumer.consumeShard(); + if (consumer.getCurrentState() == ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE) { + break; + } + Thread.sleep(50L); + } + assertThat(consumer.getCurrentState(), equalTo(ConsumerStates.ShardConsumerState.SHUTDOWN_COMPLETE)); + + assertThat(processor.getShutdownReason(), is(equalTo(ShutdownReason.TERMINATE))); + + verify(getRecordsCache).shutdown(); + + executorService.shutdown(); + executorService.awaitTermination(60, TimeUnit.SECONDS); + + String iterator = fileBasedProxy.getIterator(streamShardId, ShardIteratorType.TRIM_HORIZON.toString()); + List expectedRecords = toUserRecords(fileBasedProxy.get(iterator, numRecs).getRecords()); + verifyConsumedRecords(expectedRecords, processor.getProcessedRecords()); + file.delete(); + } + /** * Test method for {@link ShardConsumer#consumeShard()} that starts from initial position of type AT_TIMESTAMP. */ @@ -470,7 +624,7 @@ public class ShardConsumerTest { ) ); - KinesisDataFetcher dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); + dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, new SynchronousGetRecordsRetrievalStrategy(dataFetcher))); From 3c705300b5efb4e06adaab8e81f661c6225f000b Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Wed, 15 Nov 2017 10:44:10 -0800 Subject: [PATCH 27/35] Adding release notes and updating the KCL version to 1.8.8. (#268) --- README.md | 22 +++++++++++++++++++++- pom.xml | 2 +- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index c612845f..7a89922e 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,7 @@ The **Amazon Kinesis Client Library for Java** (Amazon KCL) enables Java develop 1. **Sign up for AWS** — Before you begin, you need an AWS account. For more information about creating an AWS account and retrieving your AWS credentials, see [AWS Account and Credentials][docs-signup] in the AWS SDK for Java Developer Guide. 1. **Sign up for Amazon Kinesis** — Go to the Amazon Kinesis console to sign up for the service and create an Amazon Kinesis stream. For more information, see [Create an Amazon Kinesis Stream][kinesis-guide-create] in the Amazon Kinesis Developer Guide. -1. **Minimum requirements** — To use the Amazon Kinesis Client Library, you'll need **Java 1.7+**. For more information about Amazon Kinesis Client Library requirements, see [Before You Begin][kinesis-guide-begin] in the Amazon Kinesis Developer Guide. +1. **Minimum requirements** — To use the Amazon Kinesis Client Library, you'll need **Java 1.8+**. For more information about Amazon Kinesis Client Library requirements, see [Before You Begin][kinesis-guide-begin] in the Amazon Kinesis Developer Guide. 1. **Using the Amazon Kinesis Client Library** — The best way to get familiar with the Amazon Kinesis Client Library is to read [Developing Record Consumer Applications][kinesis-guide-applications] in the Amazon Kinesis Developer Guide. ## Building from Source @@ -29,6 +29,26 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes +### Release 1.8.8 +* Fixed issues with leases losses due to `ExpiredIteratorException` in `PrefetchGetRecordsCache` and `AsynchronousFetchingStrategy`. + PrefetchGetRecordsCache will request for a new iterator and start fetching data again. + * [PR#263](https://github.com/awslabs/amazon-kinesis-client/pull/263) +* Added warning message for long running tasks. + Logging long running tasks can be enabled by setting the following configuration property: + + | Name | Default | Description | + | ---- | ------- | ----------- | + | [`logWarningForTaskAfterMillis`](https://github.com/awslabs/amazon-kinesis-client/blob/3de901ea9327370ed732af86c4d4999c8d99541c/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1367) | Not set | Milliseconds after which the logger will log a warning message for the long running task | + + * [PR#259](https://github.com/awslabs/amazon-kinesis-client/pull/259) +* Handling spurious lease renewal failures gracefully. + Added better handling of DynamoDB failures when updating leases. These failures would occur when a request to DynamoDB appeared to fail, but was actually successful. + * [PR#247](https://github.com/awslabs/amazon-kinesis-client/pull/247) +* ShutdownTask gets retried if the previous attempt on the ShutdownTask fails. + * [PR#267](https://github.com/awslabs/amazon-kinesis-client/pull/267) +* Fix for using maxRecords from `KinesisClientLibConfiguration` in `GetRecordsCache` for fetching records. + * [PR#264](https://github.com/awslabs/amazon-kinesis-client/pull/264) + ### Release 1.8.7 * Don't add a delay for synchronous requests to Kinesis Removes a delay that had been added for synchronous `GetRecords` calls to Kinesis. diff --git a/pom.xml b/pom.xml index b07d9ac0..1c521b0e 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.8-SNAPSHOT + 1.8.8 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 47e6206e8dc93bd5f7f6f3020d0bb706cd27b6c6 Mon Sep 17 00:00:00 2001 From: "Pfifer, Justin" Date: Wed, 15 Nov 2017 10:44:49 -0800 Subject: [PATCH 28/35] Release 1.8.8 of the Amazon Kinesis Client for Java Release 1.8.8 * Fixed issues with leases losses due to `ExpiredIteratorException` in `PrefetchGetRecordsCache` and `AsynchronousFetchingStrategy`. PrefetchGetRecordsCache will request for a new iterator and start fetching data again. * PR#263 * Added warning message for long running tasks. * PR#259 * Handling spurious lease renewal failures gracefully. Added better handling of DynamoDB failures when updating leases. These failures would occur when a request to DynamoDB appeared to fail, but was actually successful. * PR#247 * ShutdownTask gets retried if the previous attempt on the ShutdownTask fails. * PR#267 * Fix for using maxRecords from `KinesisClientLibConfiguration` in `GetRecordsCache` for fetching records. * PR#264 --- META-INF/MANIFEST.MF | 2 +- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index 3c6411d8..35e86907 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.7 +Bundle-Version: 1.8.8 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.8 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index f24fc574..3fb36754 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.7"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.8"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From 9074864027f2b8ff3acb2c167efb66d5f92b173b Mon Sep 17 00:00:00 2001 From: Mike Watters Date: Thu, 4 Jan 2018 09:29:44 -0700 Subject: [PATCH 29/35] [Issue 210] - Allow unexpected child shards to be ignored (#240) Allow unexpected child shards to be ignored now instead of always throwing an assertion if a child shard has an open parent, consider worker configuration before doing so. if configured to ignore such shards, do not create leases for them during shard sync. this is intended to mitigate failing worker init when processing dynamodb streams with many thousands of shards (which can happen for tables with thousands of partitions). this new behavior can be enabled by adding the following to a configuration/properties file: ``` ignoreUnexpectedChildShards = true ``` --- .../lib/worker/ConsumerStates.java | 1 + .../worker/KinesisClientLibConfiguration.java | 18 +++ .../lib/worker/ShardConsumer.java | 4 + .../lib/worker/ShardSyncTask.java | 6 +- .../lib/worker/ShardSyncTaskManager.java | 5 + .../clientlibrary/lib/worker/ShardSyncer.java | 114 +++++++++++++----- .../lib/worker/ShutdownTask.java | 6 +- .../clientlibrary/lib/worker/Worker.java | 7 +- .../KinesisClientLibConfigurationTest.java | 11 ++ .../worker/ShardSyncTaskIntegrationTest.java | 1 + .../lib/worker/ShardSyncerTest.java | 71 ++++++++++- .../lib/worker/ShutdownTaskTest.java | 6 +- 12 files changed, 212 insertions(+), 38 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java index 9121df4b..c0bdc060 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ConsumerStates.java @@ -527,6 +527,7 @@ class ConsumerStates { consumer.getStreamConfig().getStreamProxy(), consumer.getStreamConfig().getInitialPositionInStream(), consumer.isCleanupLeasesOfCompletedShards(), + consumer.isIgnoreUnexpectedChildShards(), consumer.getLeaseManager(), consumer.getTaskBackoffTimeMillis(), consumer.getGetRecordsCache()); diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 3fb36754..04d59574 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -200,6 +200,7 @@ public class KinesisClientLibConfiguration { private boolean callProcessRecordsEvenForEmptyRecordList; private long parentShardPollIntervalMillis; private boolean cleanupLeasesUponShardCompletion; + private boolean ignoreUnexpectedChildShards; private ClientConfiguration kinesisClientConfig; private ClientConfiguration dynamoDBClientConfig; private ClientConfiguration cloudWatchClientConfig; @@ -802,6 +803,13 @@ public class KinesisClientLibConfiguration { return cleanupLeasesUponShardCompletion; } + /** + * @return true if we should ignore child shards which have open parents + */ + public boolean shouldIgnoreUnexpectedChildShards() { + return ignoreUnexpectedChildShards; + } + /** * @return true if KCL should validate client provided sequence numbers with a call to Amazon Kinesis before * checkpointing for calls to {@link RecordProcessorCheckpointer#checkpoint(String)} @@ -1022,6 +1030,16 @@ public class KinesisClientLibConfiguration { return this; } + /** + * @param ignoreUnexpectedChildShards Ignore child shards with open parents. + * @return KinesisClientLibConfiguration + */ + public KinesisClientLibConfiguration withIgnoreUnexpectedChildShards( + boolean ignoreUnexpectedChildShards) { + this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; + return this; + } + /** * @param clientConfig Common client configuration used by Kinesis/DynamoDB/CloudWatch client * @return KinesisClientLibConfiguration diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 9fb8e8e9..95cc663e 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -484,6 +484,10 @@ class ShardConsumer { return cleanupLeasesOfCompletedShards; } + boolean isIgnoreUnexpectedChildShards() { + return config.shouldIgnoreUnexpectedChildShards(); + } + long getTaskBackoffTimeMillis() { return taskBackoffTimeMillis; } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTask.java index ddfb8459..5a0c3d5a 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTask.java @@ -35,6 +35,7 @@ class ShardSyncTask implements ITask { private final ILeaseManager leaseManager; private InitialPositionInStreamExtended initialPosition; private final boolean cleanupLeasesUponShardCompletion; + private final boolean ignoreUnexpectedChildShards; private final long shardSyncTaskIdleTimeMillis; private final TaskType taskType = TaskType.SHARDSYNC; @@ -49,11 +50,13 @@ class ShardSyncTask implements ITask { ILeaseManager leaseManager, InitialPositionInStreamExtended initialPositionInStream, boolean cleanupLeasesUponShardCompletion, + boolean ignoreUnexpectedChildShards, long shardSyncTaskIdleTimeMillis) { this.kinesisProxy = kinesisProxy; this.leaseManager = leaseManager; this.initialPosition = initialPositionInStream; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; + this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.shardSyncTaskIdleTimeMillis = shardSyncTaskIdleTimeMillis; } @@ -68,7 +71,8 @@ class ShardSyncTask implements ITask { ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPosition, - cleanupLeasesUponShardCompletion); + cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards); if (shardSyncTaskIdleTimeMillis > 0) { Thread.sleep(shardSyncTaskIdleTimeMillis); } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskManager.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskManager.java index c1bfae76..be62c66b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskManager.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskManager.java @@ -44,6 +44,7 @@ class ShardSyncTaskManager { private final ExecutorService executorService; private final InitialPositionInStreamExtended initialPositionInStream; private boolean cleanupLeasesUponShardCompletion; + private boolean ignoreUnexpectedChildShards; private final long shardSyncIdleTimeMillis; @@ -55,6 +56,7 @@ class ShardSyncTaskManager { * @param initialPositionInStream Initial position in stream * @param cleanupLeasesUponShardCompletion Clean up leases for shards that we've finished processing (don't wait * until they expire) + * @param ignoreUnexpectedChildShards Ignore child shards with open parents * @param shardSyncIdleTimeMillis Time between tasks to sync leases and Kinesis shards * @param metricsFactory Metrics factory * @param executorService ExecutorService to execute the shard sync tasks @@ -63,6 +65,7 @@ class ShardSyncTaskManager { final ILeaseManager leaseManager, final InitialPositionInStreamExtended initialPositionInStream, final boolean cleanupLeasesUponShardCompletion, + final boolean ignoreUnexpectedChildShards, final long shardSyncIdleTimeMillis, final IMetricsFactory metricsFactory, ExecutorService executorService) { @@ -70,6 +73,7 @@ class ShardSyncTaskManager { this.leaseManager = leaseManager; this.metricsFactory = metricsFactory; this.cleanupLeasesUponShardCompletion = cleanupLeasesUponShardCompletion; + this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.shardSyncIdleTimeMillis = shardSyncIdleTimeMillis; this.executorService = executorService; this.initialPositionInStream = initialPositionInStream; @@ -99,6 +103,7 @@ class ShardSyncTaskManager { leaseManager, initialPositionInStream, cleanupLeasesUponShardCompletion, + ignoreUnexpectedChildShards, shardSyncIdleTimeMillis), metricsFactory); future = executorService.submit(currentTask); submittedNewTask = true; diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncer.java index 52944200..2e309156 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncer.java @@ -28,6 +28,7 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.commons.lang.StringUtils; import com.amazonaws.services.kinesis.clientlibrary.exceptions.internal.KinesisClientLibIOException; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; @@ -60,9 +61,11 @@ class ShardSyncer { static synchronized void bootstrapShardLeases(IKinesisProxy kinesisProxy, ILeaseManager leaseManager, InitialPositionInStreamExtended initialPositionInStream, - boolean cleanupLeasesOfCompletedShards) + boolean cleanupLeasesOfCompletedShards, + boolean ignoreUnexpectedChildShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards); + syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards); } /** @@ -71,21 +74,28 @@ class ShardSyncer { * @param kinesisProxy * @param leaseManager * @param initialPositionInStream - * @param expectedClosedShardId If this is not null, we will assert that the shard list we get from Kinesis - * shows this shard to be closed (e.g. parent shard must be closed after a reshard operation). - * If it is open, we assume this is an race condition around a reshard event and throw - * a KinesisClientLibIOException so client can backoff and retry later. + * @param cleanupLeasesOfCompletedShards + * @param ignoreUnexpectedChildShards * @throws DependencyException * @throws InvalidStateException * @throws ProvisionedThroughputException * @throws KinesisClientLibIOException */ + static synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy, + ILeaseManager leaseManager, + InitialPositionInStreamExtended initialPositionInStream, + boolean cleanupLeasesOfCompletedShards, + boolean ignoreUnexpectedChildShards) + throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { + syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, ignoreUnexpectedChildShards); + } + static synchronized void checkAndCreateLeasesForNewShards(IKinesisProxy kinesisProxy, ILeaseManager leaseManager, InitialPositionInStreamExtended initialPositionInStream, boolean cleanupLeasesOfCompletedShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { - syncShardLeases(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards); + checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPositionInStream, cleanupLeasesOfCompletedShards, false); } /** @@ -93,11 +103,9 @@ class ShardSyncer { * * @param kinesisProxy * @param leaseManager - * @param expectedClosedShardId If this is not null, we will assert that the shard list we get from Kinesis - * does not show this shard to be open (e.g. parent shard must be closed after a reshard operation). - * If it is still open, we assume this is a race condition around a reshard event and - * throw a KinesisClientLibIOException so client can backoff and retry later. If the shard doesn't exist in - * Kinesis at all, we assume this is an old/expired shard and continue with the sync operation. + * @param initialPosition + * @param cleanupLeasesOfCompletedShards + * @param ignoreUnexpectedChildShards * @throws DependencyException * @throws InvalidStateException * @throws ProvisionedThroughputException @@ -107,18 +115,23 @@ class ShardSyncer { private static synchronized void syncShardLeases(IKinesisProxy kinesisProxy, ILeaseManager leaseManager, InitialPositionInStreamExtended initialPosition, - boolean cleanupLeasesOfCompletedShards) + boolean cleanupLeasesOfCompletedShards, + boolean ignoreUnexpectedChildShards) throws DependencyException, InvalidStateException, ProvisionedThroughputException, KinesisClientLibIOException { List shards = getShardList(kinesisProxy); LOG.debug("Num shards: " + shards.size()); Map shardIdToShardMap = constructShardIdToShardMap(shards); Map> shardIdToChildShardIdsMap = constructShardIdToChildShardIdsMap(shardIdToShardMap); - assertAllParentShardsAreClosed(shardIdToChildShardIdsMap, shardIdToShardMap); - + Set inconsistentShardIds = findInconsistentShardIds(shardIdToChildShardIdsMap, shardIdToShardMap); + if (!ignoreUnexpectedChildShards) { + assertAllParentShardsAreClosed(inconsistentShardIds); + } + List currentLeases = leaseManager.listLeases(); - - List newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition); + + List newLeasesToCreate = determineNewLeasesToCreate(shards, currentLeases, initialPosition, + inconsistentShardIds); LOG.debug("Num new leases to create: " + newLeasesToCreate.size()); for (KinesisClientLease lease : newLeasesToCreate) { long startTimeMillis = System.currentTimeMillis(); @@ -149,19 +162,37 @@ class ShardSyncer { /** Helper method to detect a race condition between fetching the shards via paginated DescribeStream calls * and a reshard operation. - * @param shardIdToChildShardIdsMap - * @param shardIdToShardMap + * @param inconsistentShardIds * @throws KinesisClientLibIOException */ - private static void assertAllParentShardsAreClosed(Map> shardIdToChildShardIdsMap, - Map shardIdToShardMap) throws KinesisClientLibIOException { + private static void assertAllParentShardsAreClosed(Set inconsistentShardIds) + throws KinesisClientLibIOException { + if (!inconsistentShardIds.isEmpty()) { + String ids = StringUtils.join(inconsistentShardIds, ' '); + 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.", + inconsistentShardIds.size(), ids)); + } + } + + /** + * Helper method to construct the list of inconsistent shards, which are open shards with non-closed ancestor + * parent(s). + * @param shardIdToChildShardIdsMap + * @param shardIdToShardMap + * @return Set of inconsistent open shard ids for shards having open parents. + */ + private static Set findInconsistentShardIds(Map> shardIdToChildShardIdsMap, + Map shardIdToShardMap) { + Set result = new HashSet(); for (String parentShardId : shardIdToChildShardIdsMap.keySet()) { Shard parentShard = shardIdToShardMap.get(parentShardId); if ((parentShardId == null) || (parentShard.getSequenceNumberRange().getEndingSequenceNumber() == null)) { - throw new KinesisClientLibIOException("Parent shardId " + parentShardId + " is not closed. " - + "This can happen due to a race condition between describeStream and a reshard operation."); + Set childShardIdsMap = shardIdToChildShardIdsMap.get(parentShardId); + result.addAll(childShardIdsMap); } } + return result; } /** @@ -296,8 +327,8 @@ class ShardSyncer { /** * Determine new leases to create and their initial checkpoint. * Note: Package level access only for testing purposes. - * - * For each open (no ending sequence number) shard that doesn't already have a lease, + * + * For each open (no ending sequence number) shard without open parents that doesn't already have a lease, * determine if it is a descendent of any shard which is or will be processed (e.g. for which a lease exists): * If so, set checkpoint of the shard to TrimHorizon and also create leases for ancestors if needed. * If not, set checkpoint of the shard to the initial position specified by the client. @@ -315,27 +346,35 @@ class ShardSyncer { * * For example: * Shard structure (each level depicts a stream segment): - * 0 1 2 3 4 5- shards till epoch 102 - * \ / \ / | | - * 6 7 4 5- shards from epoch 103 - 205 - * \ / | /\ - * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) + * 0 1 2 3 4 5 - shards till epoch 102 + * \ / \ / | | + * 6 7 4 5 - shards from epoch 103 - 205 + * \ / | / \ + * 8 4 9 10 - shards from epoch 206 (open - no ending sequenceNumber) * Current leases: (3, 4, 5) * New leases to create: (2, 6, 7, 8, 9, 10) * * The leases returned are sorted by the starting sequence number - following the same order * when persisting the leases in DynamoDB will ensure that we recover gracefully if we fail * before creating all the leases. + * + * If a shard has no existing lease, is open, and is a descendant of a parent which is still open, we ignore it + * here; this happens when the list of shards is inconsistent, which could be due to pagination delay for very + * high shard count streams (i.e., dynamodb streams for tables with thousands of partitions). This can only + * currently happen here if ignoreUnexpectedChildShards was true in syncShardleases. + * * * @param shards List of all shards in Kinesis (we'll create new leases based on this set) * @param currentLeases List of current leases * @param initialPosition One of LATEST, TRIM_HORIZON, or AT_TIMESTAMP. We'll start fetching records from that * location in the shard (when an application starts up for the first time - and there are no checkpoints). + * @param inconsistentShardIds Set of child shard ids having open parents. * @return List of new leases to create sorted by starting sequenceNumber of the corresponding shard */ static List determineNewLeasesToCreate(List shards, List currentLeases, - InitialPositionInStreamExtended initialPosition) { + InitialPositionInStreamExtended initialPosition, + Set inconsistentShardIds) { Map shardIdToNewLeaseMap = new HashMap(); Map shardIdToShardMapOfAllKinesisShards = constructShardIdToShardMap(shards); @@ -354,6 +393,8 @@ class ShardSyncer { LOG.debug("Evaluating leases for open shard " + shardId + " and its ancestors."); if (shardIdsOfCurrentLeases.contains(shardId)) { LOG.debug("Lease for shardId " + shardId + " already exists. Not creating a lease"); + } else if (inconsistentShardIds.contains(shardId)) { + LOG.info("shardId " + shardId + " is an inconsistent child. Not creating a lease"); } else { LOG.debug("Need to create a lease for shardId " + shardId); KinesisClientLease newLease = newKCLLease(shard); @@ -407,6 +448,17 @@ class ShardSyncer { return newLeasesToCreate; } + /** + * Determine new leases to create and their initial checkpoint. + * Note: Package level access only for testing purposes. + */ + static List determineNewLeasesToCreate(List shards, + List currentLeases, + InitialPositionInStreamExtended initialPosition) { + Set inconsistentShardIds = new HashSet(); + return determineNewLeasesToCreate(shards, currentLeases, initialPosition, inconsistentShardIds); + } + /** * Note: Package level access for testing purposes only. * Check if this shard is a descendant of a shard that is (or will be) processed. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java index bd40d686..a407f009 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTask.java @@ -44,6 +44,7 @@ class ShutdownTask implements ITask { private final ILeaseManager leaseManager; private final InitialPositionInStreamExtended initialPositionInStream; private final boolean cleanupLeasesOfCompletedShards; + private final boolean ignoreUnexpectedChildShards; private final TaskType taskType = TaskType.SHUTDOWN; private final long backoffTimeMillis; private final GetRecordsCache getRecordsCache; @@ -59,6 +60,7 @@ class ShutdownTask implements ITask { IKinesisProxy kinesisProxy, InitialPositionInStreamExtended initialPositionInStream, boolean cleanupLeasesOfCompletedShards, + boolean ignoreUnexpectedChildShards, ILeaseManager leaseManager, long backoffTimeMillis, GetRecordsCache getRecordsCache) { @@ -69,6 +71,7 @@ class ShutdownTask implements ITask { this.kinesisProxy = kinesisProxy; this.initialPositionInStream = initialPositionInStream; this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards; + this.ignoreUnexpectedChildShards = ignoreUnexpectedChildShards; this.leaseManager = leaseManager; this.backoffTimeMillis = backoffTimeMillis; this.getRecordsCache = getRecordsCache; @@ -127,7 +130,8 @@ class ShutdownTask implements ITask { ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, initialPositionInStream, - cleanupLeasesOfCompletedShards); + cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards); LOG.debug("Finished checking for child shards of shard " + shardInfo.getShardId()); } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index d2ea738d..c0f413be 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -407,8 +407,8 @@ public class Worker implements Runnable { this.leaseCoordinator = leaseCoordinator; this.metricsFactory = metricsFactory; this.controlServer = new ShardSyncTaskManager(streamConfig.getStreamProxy(), leaseCoordinator.getLeaseManager(), - initialPositionInStream, cleanupLeasesUponShardCompletion, shardSyncIdleTimeMillis, metricsFactory, - executorService); + initialPositionInStream, cleanupLeasesUponShardCompletion, config.shouldIgnoreUnexpectedChildShards(), + shardSyncIdleTimeMillis, metricsFactory, executorService); this.taskBackoffTimeMillis = taskBackoffTimeMillis; this.failoverTimeMillis = failoverTimeMillis; this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist; @@ -499,7 +499,8 @@ public class Worker implements Runnable { || leaseCoordinator.getLeaseManager().isLeaseTableEmpty()) { LOG.info("Syncing Kinesis shard info"); ShardSyncTask shardSyncTask = new ShardSyncTask(streamConfig.getStreamProxy(), - leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion, 0L); + leaseCoordinator.getLeaseManager(), initialPosition, cleanupLeasesUponShardCompletion, + config.shouldIgnoreUnexpectedChildShards(), 0L); result = new MetricsCollectingTaskDecorator(shardSyncTask, metricsFactory).call(); } else { LOG.info("Skipping shard sync per config setting (and lease table is not empty)"); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java index 177546db..cccbcb30 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfigurationTest.java @@ -17,6 +17,7 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; import java.util.Date; @@ -407,4 +408,14 @@ public class KinesisClientLibConfigurationTest { fail("Should not have thrown"); } } + + @Test + public void testKCLConfigurationWithIgnoreUnexpectedChildShards() { + KinesisClientLibConfiguration config = + new KinesisClientLibConfiguration("TestApplication", "TestStream", null, "TestWorker"); + // By default, unexpected child shards should not be ignored. + assertFalse(config.shouldIgnoreUnexpectedChildShards()); + config = config.withIgnoreUnexpectedChildShards(true); + assertTrue(config.shouldIgnoreUnexpectedChildShards()); + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskIntegrationTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskIntegrationTest.java index 307596e3..37a72741 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskIntegrationTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncTaskIntegrationTest.java @@ -124,6 +124,7 @@ public class ShardSyncTaskIntegrationTest { leaseManager, InitialPositionInStreamExtended.newInitialPosition(InitialPositionInStream.LATEST), false, + false, 0L); syncTask.call(); List leases = leaseManager.listLeases(); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java index b8f6ae56..2736281e 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardSyncerTest.java @@ -146,6 +146,39 @@ public class ShardSyncerTest { } } + /** + * Test determineNewLeasesToCreate() where there are no leases and no resharding operations have been performed, but one of + * the shards was marked as inconsistent. + */ + @Test + public final void testDetermineNewLeasesToCreate0Leases0Reshards1Inconsistent() { + List shards = new ArrayList(); + List currentLeases = new ArrayList(); + SequenceNumberRange sequenceRange = ShardObjectHelper.newSequenceNumberRange("342980", null); + + String shardId0 = "shardId-0"; + shards.add(ShardObjectHelper.newShard(shardId0, null, null, sequenceRange)); + + String shardId1 = "shardId-1"; + shards.add(ShardObjectHelper.newShard(shardId1, null, null, sequenceRange)); + + String shardId2 = "shardId-2"; + shards.add(ShardObjectHelper.newShard(shardId2, shardId1, null, sequenceRange)); + + Set inconsistentShardIds = new HashSet(); + inconsistentShardIds.add(shardId2); + + List newLeases = + ShardSyncer.determineNewLeasesToCreate(shards, currentLeases, INITIAL_POSITION_LATEST, inconsistentShardIds); + Assert.assertEquals(2, newLeases.size()); + Set expectedLeaseShardIds = new HashSet(); + expectedLeaseShardIds.add(shardId0); + expectedLeaseShardIds.add(shardId1); + for (KinesisClientLease lease : newLeases) { + Assert.assertTrue(expectedLeaseShardIds.contains(lease.getLeaseKey())); + } + } + /** * Test bootstrapShardLeases() starting at TRIM_HORIZON ("beginning" of stream) * @@ -296,6 +329,41 @@ public class ShardSyncerTest { dataFile.delete(); } + /** + * Test checkAndCreateLeasesForNewShards() when a parent is open and children of open parents are being ignored. + */ + @Test + public final void testCheckAndCreateLeasesForNewShardsWhenParentIsOpenAndIgnoringInconsistentChildren() + throws KinesisClientLibIOException, DependencyException, InvalidStateException, ProvisionedThroughputException, + IOException { + List shards = constructShardListForGraphA(); + Shard shard = shards.get(5); + Assert.assertEquals("shardId-5", shard.getShardId()); + SequenceNumberRange range = shard.getSequenceNumberRange(); + // shardId-5 in graph A has two children (shardId-9 and shardId-10). if shardId-5 + // is not closed, those children should be ignored when syncing shards, no leases + // should be obtained for them, and we should obtain a lease on the still-open + // parent. + range.setEndingSequenceNumber(null); + shard.setSequenceNumberRange(range); + File dataFile = KinesisLocalFileDataCreator.generateTempDataFile(shards, 2, "testBootstrap1"); + dataFile.deleteOnExit(); + IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); + ShardSyncer.checkAndCreateLeasesForNewShards(kinesisProxy, leaseManager, INITIAL_POSITION_LATEST, + cleanupLeasesOfCompletedShards, true); + List newLeases = leaseManager.listLeases(); + Set expectedLeaseShardIds = new HashSet(); + expectedLeaseShardIds.add("shardId-4"); + expectedLeaseShardIds.add("shardId-5"); + expectedLeaseShardIds.add("shardId-8"); + Assert.assertEquals(expectedLeaseShardIds.size(), newLeases.size()); + for (KinesisClientLease lease1 : newLeases) { + Assert.assertTrue(expectedLeaseShardIds.contains(lease1.getLeaseKey())); + Assert.assertEquals(ExtendedSequenceNumber.LATEST, lease1.getCheckpoint()); + } + dataFile.delete(); + } + /** * @throws KinesisClientLibIOException * @throws DependencyException @@ -586,7 +654,8 @@ public class ShardSyncerTest { dataFile.deleteOnExit(); IKinesisProxy kinesisProxy = new KinesisLocalFileProxy(dataFile.getAbsolutePath()); - ShardSyncer.bootstrapShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards); + ShardSyncer.bootstrapShardLeases(kinesisProxy, leaseManager, initialPosition, cleanupLeasesOfCompletedShards, + false); List newLeases = leaseManager.listLeases(); Assert.assertEquals(2, newLeases.size()); Set expectedLeaseShardIds = new HashSet(); diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java index 17a53137..ddf07e10 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShutdownTaskTest.java @@ -100,6 +100,7 @@ public class ShutdownTaskTest { IKinesisProxy kinesisProxy = mock(IKinesisProxy.class); ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); boolean cleanupLeasesOfCompletedShards = false; + boolean ignoreUnexpectedChildShards = false; ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -107,6 +108,7 @@ public class ShutdownTaskTest { kinesisProxy, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseManager, TASK_BACKOFF_TIME_MILLIS, getRecordsCache); @@ -126,6 +128,7 @@ public class ShutdownTaskTest { when(kinesisProxy.getShardList()).thenReturn(null); ILeaseManager leaseManager = mock(KinesisClientLeaseManager.class); boolean cleanupLeasesOfCompletedShards = false; + boolean ignoreUnexpectedChildShards = false; ShutdownTask task = new ShutdownTask(defaultShardInfo, defaultRecordProcessor, checkpointer, @@ -133,6 +136,7 @@ public class ShutdownTaskTest { kinesisProxy, INITIAL_POSITION_TRIM_HORIZON, cleanupLeasesOfCompletedShards, + ignoreUnexpectedChildShards, leaseManager, TASK_BACKOFF_TIME_MILLIS, getRecordsCache); @@ -147,7 +151,7 @@ public class ShutdownTaskTest { */ @Test public final void testGetTaskType() { - ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, null, 0, getRecordsCache); + ShutdownTask task = new ShutdownTask(null, null, null, null, null, null, false, false, null, 0, getRecordsCache); Assert.assertEquals(TaskType.SHUTDOWN, task.getTaskType()); } From db72cc15f8d708f4833ff957d5fd13d3367f3187 Mon Sep 17 00:00:00 2001 From: "Pfifer, Justin" Date: Thu, 4 Jan 2018 08:32:50 -0800 Subject: [PATCH 30/35] Advance version to 1.8.9-SNAPSHOT --- META-INF/MANIFEST.MF | 2 +- pom.xml | 2 +- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF index 35e86907..ecc2b4f0 100644 --- a/META-INF/MANIFEST.MF +++ b/META-INF/MANIFEST.MF @@ -2,7 +2,7 @@ Manifest-Version: 1.0 Bundle-ManifestVersion: 2 Bundle-Name: Amazon Kinesis Client Library for Java Bundle-SymbolicName: com.amazonaws.kinesisclientlibrary;singleton:=true -Bundle-Version: 1.8.8 +Bundle-Version: 1.8.9 Bundle-Vendor: Amazon Technologies, Inc Bundle-RequiredExecutionEnvironment: JavaSE-1.8 Require-Bundle: org.apache.commons.codec;bundle-version="1.6", diff --git a/pom.xml b/pom.xml index 1c521b0e..fea1c5df 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.8 + 1.8.9-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index 04d59574..f69ae180 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.8"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.9"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From e999c76b776557ec720193fe2e5dd6973d672009 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Fri, 12 Jan 2018 12:01:18 -0800 Subject: [PATCH 31/35] Upgraded the AWS SDK version to 1.11.261 (#281) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fea1c5df..44cce92a 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ - 1.11.218 + 1.11.261 1.0.392 libsqlite4java ${project.build.directory}/test-lib From 31fd0b58117962e99b2f793a8ff4c3c17a3d9a81 Mon Sep 17 00:00:00 2001 From: Justin Pfifer Date: Mon, 15 Jan 2018 07:52:28 -0800 Subject: [PATCH 32/35] Release 1.8.9 of the Amazon Kinesis Client for Java (#282) * Allow disabling check for the case where a child shard has an open parent shard. There is a race condition where it's possible for the a parent shard to appear open, while having child shards. This check can now be disabled by setting ignoreUnexpectedChildShards in the KinesisClientLibConfiguration to true. * PR #240 * Issue #210 * Upgraded the AWS SDK for Java to 1.11.261 * PR #281 --- README.md | 9 +++++++++ pom.xml | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 7a89922e..246b1e32 100644 --- a/README.md +++ b/README.md @@ -29,6 +29,15 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages. ## Release Notes + +### Release 1.8.9 +* Allow disabling check for the case where a child shard has an open parent shard. + There is a race condition where it's possible for the a parent shard to appear open, while having child shards. This check can now be disabled by setting [`ignoreUnexpectedChildShards`](https://github.com/awslabs/amazon-kinesis-client/blob/master/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L1037) to true. + * [PR #240](https://github.com/awslabs/amazon-kinesis-client/pull/240) + * [Issue #210](https://github.com/awslabs/amazon-kinesis-client/issues/210) +* Upgraded the AWS SDK for Java to 1.11.261 + * [PR #281](https://github.com/awslabs/amazon-kinesis-client/pull/281) + ### Release 1.8.8 * Fixed issues with leases losses due to `ExpiredIteratorException` in `PrefetchGetRecordsCache` and `AsynchronousFetchingStrategy`. PrefetchGetRecordsCache will request for a new iterator and start fetching data again. diff --git a/pom.xml b/pom.xml index 44cce92a..d430b221 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.9-SNAPSHOT + 1.8.9 The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. From 222bcdaf3f4c5873506d45a2809d5fc88db57f3e Mon Sep 17 00:00:00 2001 From: parijatsinha Date: Mon, 15 Jan 2018 14:31:04 -0800 Subject: [PATCH 33/35] Adding capability to set KinesisProxy using Worker.Builder (#274) * Added IKinesisProxy injector in Worker.Builder to allow injecting custom proxy implementations * Added unit tests for IKinesisProxy injection in Worker Builder * Revert "Added unit tests for IKinesisProxy injection in Worker Builder" This reverts commit aa944c17061b1506c5c55cf3932857b6f6086049. Reverting to undo changes to import ordering. * Added unit tests for IKinesisProxy injection in Worker Builder Re-added unit tests after reverting changes to import ordering. * Revert "Added unit tests for IKinesisProxy injection in Worker Builder" This reverts commit 91e445774beda2097788da7ffc09d04c03314a43. Reverting to refactor unit tests. * Added unit tests for Worker Builder IKinesisProxy injection validation Refactored unit tests as per comments in the pull request. * Added debug logs in KinesisLocalFileDataCreator * Revert "Added debug logs in KinesisLocalFileDataCreator" This reverts commit 1ff00d0b01d6d95a02b7ae67e542977a75b6e307. * Edited JavaDoc for Worker Builder kinesisProxy --- .../clientlibrary/lib/worker/Worker.java | 28 +++++++++++++++++-- .../clientlibrary/lib/worker/WorkerTest.java | 26 +++++++++++++++++ 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java index c0f413be..87d5cdb1 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/Worker.java @@ -32,6 +32,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -996,6 +997,11 @@ public class Worker implements Runnable { metricsFactory, execService); } + @VisibleForTesting + StreamConfig getStreamConfig() { + return streamConfig; + } + /** * Given configuration, returns appropriate metrics factory. * @@ -1073,6 +1079,7 @@ public class Worker implements Runnable { private IMetricsFactory metricsFactory; private ExecutorService execService; private ShardPrioritization shardPrioritization; + private IKinesisProxy kinesisProxy; /** * Default constructor. @@ -1192,6 +1199,19 @@ public class Worker implements Runnable { return this; } + /** + * Set KinesisProxy for the worker. + * + * @param kinesisProxy + * Sets an implementation of IKinesisProxy. + * + * @return A reference to this updated object so that method calls can be chained together. + */ + public Builder kinesisProxy(IKinesisProxy kinesisProxy) { + this.kinesisProxy = kinesisProxy; + return this; + } + /** * Build the Worker instance. * @@ -1257,13 +1277,15 @@ public class Worker implements Runnable { if (shardPrioritization == null) { shardPrioritization = new ParentsFirstShardPrioritization(1); } - + if (kinesisProxy == null) { + kinesisProxy = new KinesisProxyFactory(config.getKinesisCredentialsProvider(), kinesisClient) + .getProxy(config.getStreamName()); + } return new Worker(config.getApplicationName(), recordProcessorFactory, config, - new StreamConfig(new KinesisProxyFactory(config.getKinesisCredentialsProvider(), - kinesisClient).getProxy(config.getStreamName()), + new StreamConfig(kinesisProxy, config.getMaxRecords(), config.getIdleTimeBetweenReadsInMillis(), config.shouldCallProcessRecordsEvenForEmptyRecordList(), diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java index ce406dce..6cc7ef08 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/WorkerTest.java @@ -90,6 +90,7 @@ import com.amazonaws.services.kinesis.clientlibrary.interfaces.v2.IRecordProcess import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker.WorkerCWMetricsFactory; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker.WorkerThreadPoolExecutor; import com.amazonaws.services.kinesis.clientlibrary.proxies.IKinesisProxy; +import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisLocalFileProxy; import com.amazonaws.services.kinesis.clientlibrary.proxies.util.KinesisLocalFileDataCreator; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; @@ -1474,6 +1475,31 @@ public class WorkerTest { } + @Test + public void testBuilderWithDefaultKinesisProxy() { + IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); + Worker worker = new Worker.Builder() + .recordProcessorFactory(recordProcessorFactory) + .config(config) + .build(); + Assert.assertNotNull(worker.getStreamConfig().getStreamProxy()); + Assert.assertTrue(worker.getStreamConfig().getStreamProxy() instanceof KinesisProxy); + } + + @Test + public void testBuilderWhenKinesisProxyIsSet() { + IRecordProcessorFactory recordProcessorFactory = mock(IRecordProcessorFactory.class); + // Create an instance of KinesisLocalFileProxy for injection and validation + IKinesisProxy kinesisProxy = mock(KinesisLocalFileProxy.class); + Worker worker = new Worker.Builder() + .recordProcessorFactory(recordProcessorFactory) + .config(config) + .kinesisProxy(kinesisProxy) + .build(); + Assert.assertNotNull(worker.getStreamConfig().getStreamProxy()); + Assert.assertTrue(worker.getStreamConfig().getStreamProxy() instanceof KinesisLocalFileProxy); + } + private abstract class InjectableWorker extends Worker { InjectableWorker(String applicationName, IRecordProcessorFactory recordProcessorFactory, KinesisClientLibConfiguration config, StreamConfig streamConfig, From 71124e40550208cae7e2117415914d601036c8f2 Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Tue, 16 Jan 2018 15:58:26 -0800 Subject: [PATCH 34/35] Updating version to 1.8.10. (#283) --- pom.xml | 2 +- .../clientlibrary/lib/worker/KinesisClientLibConfiguration.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index d430b221..45327898 100644 --- a/pom.xml +++ b/pom.xml @@ -6,7 +6,7 @@ amazon-kinesis-client jar Amazon Kinesis Client Library for Java - 1.8.9 + 1.8.10-SNAPSHOT The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data from Amazon Kinesis. diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java index f69ae180..bd33fc90 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java @@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration { /** * User agent set when Amazon Kinesis Client Library makes AWS requests. */ - public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.9"; + public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.10"; /** * KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls From e65e56380b6accd96b96c423162859c20bca04fc Mon Sep 17 00:00:00 2001 From: Sahil Palvia Date: Mon, 22 Jan 2018 10:37:46 -0800 Subject: [PATCH 35/35] Fixing issue with NullMetrics warning messages (#284) Fixes #48 * Fixing issue with NullMetrics warning messages when trying to checkpoint on a separate thread. * Adding testing to validate the MetricsScope setting during checkpoiniting. --- .../worker/RecordProcessorCheckpointer.java | 47 +++++++--- .../lib/worker/ShardConsumer.java | 3 +- .../kinesis/metrics/impl/MetricsHelper.java | 11 ++- .../RecordProcessorCheckpointerTest.java | 87 ++++++++++++++----- .../lib/worker/ShardConsumerTest.java | 9 +- 5 files changed, 116 insertions(+), 41 deletions(-) diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java index 72e18d73..8e3dfd73 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointer.java @@ -14,6 +14,9 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; +import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingScope; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -50,6 +53,8 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer { private SequenceNumberValidator sequenceNumberValidator; private ExtendedSequenceNumber sequenceNumberAtShardEnd; + + private IMetricsFactory metricsFactory; /** * Only has package level access, since only the Amazon Kinesis Client Library should be creating these. @@ -59,10 +64,12 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer { */ RecordProcessorCheckpointer(ShardInfo shardInfo, ICheckpoint checkpoint, - SequenceNumberValidator validator) { + SequenceNumberValidator validator, + IMetricsFactory metricsFactory) { this.shardInfo = shardInfo; this.checkpoint = checkpoint; this.sequenceNumberValidator = validator; + this.metricsFactory = metricsFactory; } /** @@ -283,21 +290,33 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer { // just checkpoint at SHARD_END checkpointToRecord = ExtendedSequenceNumber.SHARD_END; } + + boolean unsetMetrics = false; // Don't checkpoint a value we already successfully checkpointed - if (extendedSequenceNumber != null && !extendedSequenceNumber.equals(lastCheckpointValue)) { - try { - if (LOG.isDebugEnabled()) { - LOG.debug("Setting " + shardInfo.getShardId() + ", token " + shardInfo.getConcurrencyToken() - + " checkpoint to " + checkpointToRecord); + try { + if (!MetricsHelper.isMetricsScopePresent()) { + MetricsHelper.setMetricsScope(new ThreadSafeMetricsDelegatingScope(metricsFactory.createMetrics())); + unsetMetrics = true; + } + if (extendedSequenceNumber != null && !extendedSequenceNumber.equals(lastCheckpointValue)) { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Setting " + shardInfo.getShardId() + ", token " + shardInfo.getConcurrencyToken() + + " checkpoint to " + checkpointToRecord); + } + checkpoint.setCheckpoint(shardInfo.getShardId(), checkpointToRecord, shardInfo.getConcurrencyToken()); + lastCheckpointValue = checkpointToRecord; + } catch (ThrottlingException | ShutdownException | InvalidStateException + | KinesisClientLibDependencyException e) { + throw e; + } catch (KinesisClientLibException e) { + LOG.warn("Caught exception setting checkpoint.", e); + throw new KinesisClientLibDependencyException("Caught exception while checkpointing", e); } - checkpoint.setCheckpoint(shardInfo.getShardId(), checkpointToRecord, shardInfo.getConcurrencyToken()); - lastCheckpointValue = checkpointToRecord; - } catch (ThrottlingException | ShutdownException | InvalidStateException - | KinesisClientLibDependencyException e) { - throw e; - } catch (KinesisClientLibException e) { - LOG.warn("Caught exception setting checkpoint.", e); - throw new KinesisClientLibDependencyException("Caught exception while checkpointing", e); + } + } finally { + if (unsetMetrics) { + MetricsHelper.unsetMetricsScope(); } } } diff --git a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java index 95cc663e..4a001b9b 100644 --- a/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java +++ b/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumer.java @@ -170,7 +170,8 @@ class ShardConsumer { new SequenceNumberValidator( streamConfig.getStreamProxy(), shardInfo.getShardId(), - streamConfig.shouldValidateSequenceNumberBeforeCheckpointing())), + streamConfig.shouldValidateSequenceNumberBeforeCheckpointing()), + metricsFactory), leaseManager, parentShardPollIntervalMillis, cleanupLeasesOfCompletedShards, diff --git a/src/main/java/com/amazonaws/services/kinesis/metrics/impl/MetricsHelper.java b/src/main/java/com/amazonaws/services/kinesis/metrics/impl/MetricsHelper.java index 4599fbaa..bf104cff 100644 --- a/src/main/java/com/amazonaws/services/kinesis/metrics/impl/MetricsHelper.java +++ b/src/main/java/com/amazonaws/services/kinesis/metrics/impl/MetricsHelper.java @@ -72,13 +72,22 @@ public class MetricsHelper { * @param scope */ public static void setMetricsScope(IMetricsScope scope) { - if (currentScope.get() != null) { + if (isMetricsScopePresent()) { throw new RuntimeException(String.format( "Metrics scope is already set for the current thread %s", Thread.currentThread().getName())); } currentScope.set(scope); } + /** + * Checks if current metricsscope is present or not. + * + * @return true if metrics scope is present, else returns false + */ + public static boolean isMetricsScopePresent() { + return currentScope.get() != null; + } + /** * Unsets the metrics scope for the current thread. */ diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java index 31a1e184..7e637457 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/RecordProcessorCheckpointerTest.java @@ -14,6 +14,13 @@ */ package com.amazonaws.services.kinesis.clientlibrary.lib.worker; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; @@ -23,7 +30,10 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; import org.mockito.Mockito; +import org.mockito.runners.MockitoJUnitRunner; import com.amazonaws.services.kinesis.clientlibrary.interfaces.ICheckpoint; import com.amazonaws.services.kinesis.clientlibrary.interfaces.IPreparedCheckpointer; @@ -31,15 +41,15 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.InMemoryCheck import com.amazonaws.services.kinesis.clientlibrary.lib.checkpoint.SentinelCheckpoint; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper; +import com.amazonaws.services.kinesis.metrics.impl.NullMetricsScope; +import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsFactory; import com.amazonaws.services.kinesis.model.Record; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Matchers.anyString; - /** * */ +@RunWith(MockitoJUnitRunner.class) public class RecordProcessorCheckpointerTest { private String startingSequenceNumber = "13"; private ExtendedSequenceNumber startingExtendedSequenceNumber = new ExtendedSequenceNumber(startingSequenceNumber); @@ -48,6 +58,9 @@ public class RecordProcessorCheckpointerTest { private ShardInfo shardInfo; private SequenceNumberValidator sequenceNumberValidator; private String shardId = "shardId-123"; + + @Mock + IMetricsFactory metricsFactory; /** * @throws java.lang.Exception @@ -78,7 +91,7 @@ public class RecordProcessorCheckpointerTest { public final void testCheckpoint() throws Exception { // First call to checkpoint RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, null); + new RecordProcessorCheckpointer(shardInfo, checkpoint, null, metricsFactory); processingCheckpointer.setLargestPermittedCheckpointValue(startingExtendedSequenceNumber); processingCheckpointer.checkpoint(); Assert.assertEquals(startingExtendedSequenceNumber, checkpoint.getCheckpoint(shardId)); @@ -98,7 +111,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testCheckpointRecord() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5025"); Record record = new Record().withSequenceNumber("5025"); @@ -114,7 +127,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testCheckpointSubRecord() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5030"); Record record = new Record().withSequenceNumber("5030"); @@ -131,7 +144,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testCheckpointSequenceNumber() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5035"); processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); @@ -146,7 +159,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testCheckpointExtendedSequenceNumber() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5040"); processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); @@ -162,7 +175,7 @@ public class RecordProcessorCheckpointerTest { public final void testPrepareCheckpoint() throws Exception { // First call to checkpoint RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber sequenceNumber1 = new ExtendedSequenceNumber("5001"); @@ -193,7 +206,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testPrepareCheckpointRecord() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5025"); Record record = new Record().withSequenceNumber("5025"); @@ -218,7 +231,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testPrepareCheckpointSubRecord() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5030"); Record record = new Record().withSequenceNumber("5030"); @@ -244,7 +257,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testPrepareCheckpointSequenceNumber() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5035"); processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); @@ -268,7 +281,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testPrepareCheckpointExtendedSequenceNumber() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber("5040"); processingCheckpointer.setLargestPermittedCheckpointValue(extendedSequenceNumber); @@ -291,7 +304,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testMultipleOutstandingCheckpointersHappyCase() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); processingCheckpointer.setLargestPermittedCheckpointValue(new ExtendedSequenceNumber("6040")); @@ -323,7 +336,7 @@ public class RecordProcessorCheckpointerTest { @Test public final void testMultipleOutstandingCheckpointersOutOfOrder() throws Exception { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, sequenceNumberValidator, metricsFactory); processingCheckpointer.setInitialCheckpointValue(startingExtendedSequenceNumber); processingCheckpointer.setLargestPermittedCheckpointValue(new ExtendedSequenceNumber("7040")); @@ -358,7 +371,7 @@ public class RecordProcessorCheckpointerTest { */ @Test public final void testUpdate() throws Exception { - RecordProcessorCheckpointer checkpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, null); + RecordProcessorCheckpointer checkpointer = new RecordProcessorCheckpointer(shardInfo, checkpoint, null, metricsFactory); ExtendedSequenceNumber sequenceNumber = new ExtendedSequenceNumber("10"); checkpointer.setLargestPermittedCheckpointValue(sequenceNumber); @@ -379,7 +392,7 @@ public class RecordProcessorCheckpointerTest { SequenceNumberValidator validator = mock(SequenceNumberValidator.class); Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator, metricsFactory); // Several checkpoints we're gonna hit ExtendedSequenceNumber tooSmall = new ExtendedSequenceNumber("2"); @@ -467,7 +480,7 @@ public class RecordProcessorCheckpointerTest { SequenceNumberValidator validator = mock(SequenceNumberValidator.class); Mockito.doNothing().when(validator).validateSequenceNumber(anyString()); RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator, metricsFactory); // Several checkpoints we're gonna hit ExtendedSequenceNumber tooSmall = new ExtendedSequenceNumber("2"); @@ -595,7 +608,7 @@ public class RecordProcessorCheckpointerTest { for (LinkedHashMap testPlan : getMixedCallsTestPlan()) { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator, metricsFactory); testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.CHECKPOINTER); } } @@ -615,7 +628,7 @@ public class RecordProcessorCheckpointerTest { for (LinkedHashMap testPlan : getMixedCallsTestPlan()) { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator, metricsFactory); testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.PREPARED_CHECKPOINTER); } } @@ -636,7 +649,7 @@ public class RecordProcessorCheckpointerTest { for (LinkedHashMap testPlan : getMixedCallsTestPlan()) { RecordProcessorCheckpointer processingCheckpointer = - new RecordProcessorCheckpointer(shardInfo, checkpoint, validator); + new RecordProcessorCheckpointer(shardInfo, checkpoint, validator, metricsFactory); testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.PREPARE_THEN_CHECKPOINTER); } } @@ -785,4 +798,34 @@ public class RecordProcessorCheckpointerTest { Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint()); } } + + @Test + public final void testUnsetMetricsScopeDuringCheckpointing() throws Exception { + // First call to checkpoint + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, null, metricsFactory); + ExtendedSequenceNumber sequenceNumber = new ExtendedSequenceNumber("5019"); + processingCheckpointer.setLargestPermittedCheckpointValue(sequenceNumber); + processingCheckpointer.checkpoint(); + Assert.assertEquals(sequenceNumber, checkpoint.getCheckpoint(shardId)); + verify(metricsFactory).createMetrics(); + Assert.assertFalse(MetricsHelper.isMetricsScopePresent()); + } + + @Test + public final void testSetMetricsScopeDuringCheckpointing() throws Exception { + // First call to checkpoint + RecordProcessorCheckpointer processingCheckpointer = + new RecordProcessorCheckpointer(shardInfo, checkpoint, null, metricsFactory); + NullMetricsScope scope = new NullMetricsScope(); + MetricsHelper.setMetricsScope(scope); + ExtendedSequenceNumber sequenceNumber = new ExtendedSequenceNumber("5019"); + processingCheckpointer.setLargestPermittedCheckpointValue(sequenceNumber); + processingCheckpointer.checkpoint(); + Assert.assertEquals(sequenceNumber, checkpoint.getCheckpoint(shardId)); + verify(metricsFactory, never()).createMetrics(); + Assert.assertTrue(MetricsHelper.isMetricsScopePresent()); + assertEquals(scope, MetricsHelper.getMetricsScope()); + MetricsHelper.unsetMetricsScope(); + } } diff --git a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java index 8a91c6e6..216d59cd 100644 --- a/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java +++ b/src/test/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/ShardConsumerTest.java @@ -342,7 +342,8 @@ public class ShardConsumerTest { streamConfig.getStreamProxy(), shardInfo.getShardId(), streamConfig.shouldValidateSequenceNumberBeforeCheckpointing() - ) + ), + metricsFactory ); dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo); @@ -493,7 +494,8 @@ public class ShardConsumerTest { streamConfig.getStreamProxy(), shardInfo.getShardId(), streamConfig.shouldValidateSequenceNumberBeforeCheckpointing() - ) + ), + metricsFactory ); ShardConsumer consumer = @@ -621,7 +623,8 @@ public class ShardConsumerTest { streamConfig.getStreamProxy(), shardInfo.getShardId(), streamConfig.shouldValidateSequenceNumberBeforeCheckpointing() - ) + ), + metricsFactory ); dataFetcher = new KinesisDataFetcher(streamConfig.getStreamProxy(), shardInfo);