From 1ec0b656c9c77a355640b9c32c1cbca0d592e546 Mon Sep 17 00:00:00 2001 From: Walid Baruni Date: Tue, 15 Aug 2017 12:56:32 -0700 Subject: [PATCH 01/11] 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 02/11] 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 03/11] 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 04/11] 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 05/11] 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 06/11] 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 07/11] 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 08/11] 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 09/11] 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 10/11] 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 11/11] 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