Merge remote-tracking branch 'upstream/master' into spurious-update-fix
This commit is contained in:
commit
c978d3c223
62 changed files with 2753 additions and 405 deletions
|
|
@ -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.5
|
||||
Bundle-Vendor: Amazon Technologies, Inc
|
||||
Bundle-RequiredExecutionEnvironment: JavaSE-1.7
|
||||
Require-Bundle: org.apache.commons.codec;bundle-version="1.6",
|
||||
|
|
|
|||
33
README.md
33
README.md
|
|
@ -29,6 +29,39 @@ 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.
|
||||
* [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.
|
||||
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.
|
||||
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/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/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)
|
||||
|
||||
### 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.
|
||||
|
|
|
|||
4
pom.xml
4
pom.xml
|
|
@ -6,7 +6,7 @@
|
|||
<artifactId>amazon-kinesis-client</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<name>Amazon Kinesis Client Library for Java</name>
|
||||
<version>1.8.2-SNAPSHOT</version>
|
||||
<version>1.8.5</version>
|
||||
<description>The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
|
||||
from Amazon Kinesis.
|
||||
</description>
|
||||
|
|
@ -25,7 +25,7 @@
|
|||
</licenses>
|
||||
|
||||
<properties>
|
||||
<aws-java-sdk.version>1.11.171</aws-java-sdk.version>
|
||||
<aws-java-sdk.version>1.11.198</aws-java-sdk.version>
|
||||
<sqlite4java.version>1.0.392</sqlite4java.version>
|
||||
<sqlite4java.native>libsqlite4java</sqlite4java.native>
|
||||
<sqlite4java.libpath>${project.build.directory}/test-lib</sqlite4java.libpath>
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,143 @@
|
|||
/*
|
||||
* 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;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
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 java.util.function.Supplier;
|
||||
|
||||
import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper;
|
||||
import com.amazonaws.services.kinesis.metrics.impl.ThreadSafeMetricsDelegatingScope;
|
||||
import com.amazonaws.services.kinesis.model.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 Supplier<CompletionService<DataFetcherResult>> completionServiceSupplier;
|
||||
|
||||
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, Supplier<CompletionService<DataFetcherResult>> completionServiceSupplier,
|
||||
String shardId) {
|
||||
this.dataFetcher = dataFetcher;
|
||||
this.executorService = executorService;
|
||||
this.retryGetRecordsInSeconds = retryGetRecordsInSeconds;
|
||||
this.completionServiceSupplier = completionServiceSupplier;
|
||||
this.shardId = shardId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GetRecordsResult getRecords(final int maxRecords) {
|
||||
if (executorService.isShutdown()) {
|
||||
throw new IllegalStateException("Strategy has been shutdown");
|
||||
}
|
||||
GetRecordsResult result = null;
|
||||
CompletionService<DataFetcherResult> completionService = completionServiceSupplier.get();
|
||||
Set<Future<DataFetcherResult>> futures = new HashSet<>();
|
||||
Callable<DataFetcherResult> 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<DataFetcherResult> resultFuture = completionService.poll(retryGetRecordsInSeconds,
|
||||
TimeUnit.SECONDS);
|
||||
if (resultFuture != null) {
|
||||
//
|
||||
// Fix to ensure that we only let the shard iterator advance when we intend to return the result
|
||||
// to the caller. This ensures that the shard iterator is consistently advance in step with
|
||||
// what the caller sees.
|
||||
//
|
||||
result = resultFuture.get().accept();
|
||||
break;
|
||||
}
|
||||
} catch (ExecutionException e) {
|
||||
log.error("ExecutionException thrown while trying to get records", e);
|
||||
} catch (InterruptedException e) {
|
||||
log.error("Thread was interrupted", e);
|
||||
break;
|
||||
}
|
||||
}
|
||||
futures.forEach(f -> f.cancel(true));
|
||||
return result;
|
||||
}
|
||||
|
||||
private Callable<DataFetcherResult> 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());
|
||||
}
|
||||
}
|
||||
|
|
@ -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.getGetRecordsRetrievalStrategy());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -513,7 +516,8 @@ class ConsumerStates {
|
|||
consumer.getStreamConfig().getStreamProxy(),
|
||||
consumer.getStreamConfig().getInitialPositionInStream(),
|
||||
consumer.isCleanupLeasesOfCompletedShards(), consumer.getLeaseManager(),
|
||||
consumer.getTaskBackoffTimeMillis());
|
||||
consumer.getTaskBackoffTimeMillis(),
|
||||
consumer.getGetRecordsRetrievalStrategy());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
@ -0,0 +1,66 @@
|
|||
/*
|
||||
* 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;
|
||||
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
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* 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 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();
|
||||
}
|
||||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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.5";
|
||||
|
||||
/**
|
||||
* KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls
|
||||
|
|
@ -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,10 +218,17 @@ 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<Integer> timeoutInSeconds = Optional.empty();
|
||||
|
||||
@Getter
|
||||
private Optional<Integer> retryGetRecordsInSeconds = Optional.empty();
|
||||
|
||||
@Getter
|
||||
private Optional<Integer> maxGetRecordsThreadPool = Optional.empty();
|
||||
|
||||
@Getter
|
||||
private int maxLeaseRenewalThreads = DEFAULT_MAX_LEASE_RENEWAL_THREADS;
|
||||
|
||||
|
|
@ -262,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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -297,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
|
||||
|
|
@ -322,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,
|
||||
|
|
@ -330,7 +345,7 @@ public class KinesisClientLibConfiguration {
|
|||
shardSyncIntervalMillis, cleanupTerminatedShardsBeforeExpiry,
|
||||
kinesisClientConfig, dynamoDBClientConfig, cloudWatchClientConfig,
|
||||
taskBackoffTimeMillis, metricsBufferTimeMillis, metricsMaxQueueSize,
|
||||
validateSequenceNumberBeforeCheckpointing, regionName);
|
||||
validateSequenceNumberBeforeCheckpointing, regionName, shutdownGraceMillis);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -392,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);
|
||||
|
|
@ -402,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;
|
||||
|
|
@ -438,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
|
||||
|
|
@ -725,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
|
||||
|
|
@ -1111,6 +1137,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
|
||||
*/
|
||||
|
|
@ -1118,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;
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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<KinesisClientLea
|
|||
}
|
||||
|
||||
lease.setCheckpoint(checkpoint);
|
||||
lease.setPendingCheckpoint(null);
|
||||
lease.setOwnerSwitchesSinceCheckpoint(0L);
|
||||
|
||||
return updateLease(lease, concurrencyToken);
|
||||
|
|
@ -198,6 +201,75 @@ class KinesisClientLibLeaseCoordinator extends LeaseCoordinator<KinesisClientLea
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Records pending checkpoint for a shard. Does not modify checkpoint or ownerSwitchesSinceCheckpoint.
|
||||
*
|
||||
* @param shardId shardId to update the checkpoint for
|
||||
* @param pendingCheckpoint pending checkpoint value to set, not null
|
||||
* @param concurrencyToken obtained by calling Lease.getConcurrencyToken for a currently held lease
|
||||
*
|
||||
* @return true if setting the pending checkpoint succeeded, false otherwise
|
||||
*
|
||||
* @throws InvalidStateException if lease table does not exist
|
||||
* @throws ProvisionedThroughputException if DynamoDB update fails due to lack of capacity
|
||||
* @throws DependencyException if DynamoDB update fails in an unexpected way
|
||||
*/
|
||||
boolean prepareCheckpoint(String shardId, ExtendedSequenceNumber pendingCheckpoint, UUID concurrencyToken)
|
||||
throws DependencyException, InvalidStateException, ProvisionedThroughputException {
|
||||
KinesisClientLease lease = getCurrentlyHeldLease(shardId);
|
||||
if (lease == null) {
|
||||
LOG.info(String.format(
|
||||
"Worker %s could not prepare checkpoint for shard %s because it does not hold the lease",
|
||||
getWorkerIdentifier(),
|
||||
shardId));
|
||||
return false;
|
||||
}
|
||||
|
||||
lease.setPendingCheckpoint(Objects.requireNonNull(pendingCheckpoint, "pendingCheckpoint should not be null"));
|
||||
return updateLease(lease, concurrencyToken);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void prepareCheckpoint(String shardId,
|
||||
ExtendedSequenceNumber pendingCheckpointValue,
|
||||
String concurrencyToken) throws KinesisClientLibException {
|
||||
try {
|
||||
boolean wasSuccessful =
|
||||
prepareCheckpoint(shardId, pendingCheckpointValue, UUID.fromString(concurrencyToken));
|
||||
if (!wasSuccessful) {
|
||||
throw new ShutdownException(
|
||||
"Can't prepare checkpoint - instance doesn't hold the lease for this shard");
|
||||
}
|
||||
} catch (ProvisionedThroughputException e) {
|
||||
throw new ThrottlingException("Got throttled while preparing checkpoint.", e);
|
||||
} catch (InvalidStateException e) {
|
||||
String message = "Unable to prepare checkpoint for shardId " + shardId;
|
||||
LOG.error(message, e);
|
||||
throw new com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException(message, e);
|
||||
} catch (DependencyException e) {
|
||||
throw new KinesisClientLibDependencyException("Unable to prepare checkpoint for shardId " + shardId, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public Checkpoint getCheckpointObject(String shardId) throws KinesisClientLibException {
|
||||
try {
|
||||
KinesisClientLease lease = leaseManager.getLease(shardId);
|
||||
return new Checkpoint(lease.getCheckpoint(), lease.getPendingCheckpoint());
|
||||
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
|
||||
String message = "Unable to fetch checkpoint for shardId " + shardId;
|
||||
LOG.error(message, e);
|
||||
throw new KinesisClientLibIOException(message, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Current shard/lease assignments
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -0,0 +1,65 @@
|
|||
/*
|
||||
* 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;
|
||||
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.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 class PreparedCheckpointer implements IPreparedCheckpointer {
|
||||
|
||||
private final ExtendedSequenceNumber pendingCheckpointSequenceNumber;
|
||||
private final IRecordProcessorCheckpointer checkpointer;
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param pendingCheckpointSequenceNumber sequence number to checkpoint at
|
||||
* @param checkpointer checkpointer to use
|
||||
*/
|
||||
public PreparedCheckpointer(ExtendedSequenceNumber pendingCheckpointSequenceNumber,
|
||||
IRecordProcessorCheckpointer checkpointer) {
|
||||
this.pendingCheckpointSequenceNumber = pendingCheckpointSequenceNumber;
|
||||
this.checkpointer = checkpointer;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ExtendedSequenceNumber getPendingCheckpoint() {
|
||||
return pendingCheckpointSequenceNumber;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public void checkpoint()
|
||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException,
|
||||
IllegalArgumentException {
|
||||
checkpointer.checkpoint(pendingCheckpointSequenceNumber.getSequenceNumber(),
|
||||
pendingCheckpointSequenceNumber.getSubSequenceNumber());
|
||||
}
|
||||
}
|
||||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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,8 @@ class ProcessTask implements ITask {
|
|||
private final Shard shard;
|
||||
private final ThrottlingReporter throttlingReporter;
|
||||
|
||||
private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
||||
/**
|
||||
* @param shardInfo
|
||||
* contains information about the shard
|
||||
|
|
@ -75,13 +78,17 @@ class ProcessTask implements ITask {
|
|||
* Kinesis data fetcher (used to fetch records from Kinesis)
|
||||
* @param backoffTimeMillis
|
||||
* backoff time when catching exceptions
|
||||
* @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) {
|
||||
long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) {
|
||||
this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis,
|
||||
skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()));
|
||||
new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()),
|
||||
getRecordsRetrievalStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -103,7 +110,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 +120,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 +376,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
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
@ -23,6 +23,7 @@ import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibE
|
|||
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.interfaces.IRecordProcessorCheckpointer;
|
||||
import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber;
|
||||
import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
|
||||
|
|
@ -127,7 +128,7 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer {
|
|||
* If there is a last checkpoint value, we want to check both the lower and upper bound.
|
||||
*/
|
||||
ExtendedSequenceNumber newCheckpoint = new ExtendedSequenceNumber(sequenceNumber, subSequenceNumber);
|
||||
if ((lastCheckpointValue.compareTo(newCheckpoint) <= 0)
|
||||
if ((lastCheckpointValue == null || lastCheckpointValue.compareTo(newCheckpoint) <= 0)
|
||||
&& newCheckpoint.compareTo(largestPermittedCheckpointValue) <= 0) {
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
|
@ -144,6 +145,82 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public synchronized IPreparedCheckpointer prepareCheckpoint()
|
||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||
return this.prepareCheckpoint(
|
||||
this.largestPermittedCheckpointValue.getSequenceNumber(),
|
||||
this.largestPermittedCheckpointValue.getSubSequenceNumber());
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public synchronized IPreparedCheckpointer prepareCheckpoint(Record record)
|
||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||
if (record == null) {
|
||||
throw new IllegalArgumentException("Could not prepare checkpoint a null record");
|
||||
} else if (record instanceof UserRecord) {
|
||||
return prepareCheckpoint(record.getSequenceNumber(), ((UserRecord) record).getSubSequenceNumber());
|
||||
} else {
|
||||
return prepareCheckpoint(record.getSequenceNumber(), 0);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public synchronized IPreparedCheckpointer prepareCheckpoint(String sequenceNumber)
|
||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||
return prepareCheckpoint(sequenceNumber, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public synchronized IPreparedCheckpointer prepareCheckpoint(String sequenceNumber, long subSequenceNumber)
|
||||
throws KinesisClientLibDependencyException, InvalidStateException, ThrottlingException, ShutdownException {
|
||||
|
||||
if (subSequenceNumber < 0) {
|
||||
throw new IllegalArgumentException("Could not checkpoint at invalid, negative subsequence number "
|
||||
+ subSequenceNumber);
|
||||
}
|
||||
|
||||
// throws exception if sequence number shouldn't be checkpointed for this shard
|
||||
sequenceNumberValidator.validateSequenceNumber(sequenceNumber);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Validated prepareCheckpoint sequence number " + sequenceNumber + " for " + shardInfo.getShardId()
|
||||
+ ", token " + shardInfo.getConcurrencyToken());
|
||||
}
|
||||
/*
|
||||
* If there isn't a last checkpoint value, we only care about checking the upper bound.
|
||||
* If there is a last checkpoint value, we want to check both the lower and upper bound.
|
||||
*/
|
||||
ExtendedSequenceNumber pendingCheckpoint = new ExtendedSequenceNumber(sequenceNumber, subSequenceNumber);
|
||||
if ((lastCheckpointValue == null || lastCheckpointValue.compareTo(pendingCheckpoint) <= 0)
|
||||
&& pendingCheckpoint.compareTo(largestPermittedCheckpointValue) <= 0) {
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Preparing checkpoint " + shardInfo.getShardId()
|
||||
+ ", token " + shardInfo.getConcurrencyToken()
|
||||
+ " at specific extended sequence number " + pendingCheckpoint);
|
||||
}
|
||||
return doPrepareCheckpoint(pendingCheckpoint);
|
||||
} else {
|
||||
throw new IllegalArgumentException(String.format(
|
||||
"Could not prepare checkpoint at extended sequence number %s as it did not fall into acceptable "
|
||||
+ "range between the last checkpoint %s and the greatest extended sequence number passed "
|
||||
+ "to this record processor %s",
|
||||
pendingCheckpoint, this.lastCheckpointValue, this.largestPermittedCheckpointValue));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the lastCheckpointValue
|
||||
*/
|
||||
|
|
@ -165,7 +242,7 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param checkpoint the checkpoint value to set
|
||||
* @param largestPermittedCheckpointValue the largest permitted checkpoint
|
||||
*/
|
||||
synchronized void setLargestPermittedCheckpointValue(ExtendedSequenceNumber largestPermittedCheckpointValue) {
|
||||
this.largestPermittedCheckpointValue = largestPermittedCheckpointValue;
|
||||
|
|
@ -224,4 +301,58 @@ class RecordProcessorCheckpointer implements IRecordProcessorCheckpointer {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method stores the given sequenceNumber as a pending checkpooint in the lease table without overwriting the
|
||||
* current checkpoint, then returns a PreparedCheckpointer that is ready to checkpoint at the given sequence number.
|
||||
*
|
||||
* This method does not advance lastCheckpointValue, but calls to PreparedCheckpointer.checkpoint() on the returned
|
||||
* objects do. This allows customers to 'discard' prepared checkpoints by calling any of the 4 checkpoint methods on
|
||||
* this class before calling PreparedCheckpointer.checkpoint(). Some examples:
|
||||
*
|
||||
* 1) prepareCheckpoint(snA); checkpoint(snB). // this works regardless of whether snA or snB is bigger. It discards
|
||||
* the prepared checkpoint at snA.
|
||||
* 2) prepareCheckpoint(snA); prepareCheckpoint(snB). // this works regardless of whether snA or snB is bigger. It
|
||||
* replaces the preparedCheckpoint at snA with a new one at snB.
|
||||
* 3) checkpointerA = prepareCheckpoint(snA); checkpointerB = prepareCheckpoint(snB); checkpointerB.checkpoint();
|
||||
* checkpointerA.checkpoint(); // This replaces the prepared checkpoint at snA with a new one at snB, then
|
||||
* checkpoints at snB regardless of whether snA or snB is bigger. The checkpoint at snA only succeeds if snA > 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;
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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;
|
||||
|
||||
|
|
@ -58,6 +60,20 @@ class ShardConsumer {
|
|||
private long currentTaskSubmitTime;
|
||||
private Future<TaskResult> future;
|
||||
|
||||
@Getter
|
||||
private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
||||
private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher,
|
||||
Optional<Integer> retryGetRecordsInSeconds,
|
||||
Optional<Integer> maxGetRecordsThreadPool,
|
||||
ShardInfo shardInfo) {
|
||||
Optional<GetRecordsRetrievalStrategy> 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
|
||||
* much coordination/synchronization to handle concurrent reads/updates.
|
||||
|
|
@ -93,6 +109,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<KinesisClientLease> leaseManager,
|
||||
long parentShardPollIntervalMillis,
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
ExecutorService executorService,
|
||||
IMetricsFactory metricsFactory,
|
||||
long backoffTimeMillis,
|
||||
boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
Optional<Integer> retryGetRecordsInSeconds,
|
||||
Optional<Integer> maxGetRecordsThreadPool) {
|
||||
this.streamConfig = streamConfig;
|
||||
this.recordProcessor = recordProcessor;
|
||||
this.executorService = executorService;
|
||||
|
|
@ -111,6 +159,7 @@ class ShardConsumer {
|
|||
this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards;
|
||||
this.taskBackoffTimeMillis = backoffTimeMillis;
|
||||
this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
this.getRecordsRetrievalStrategy = makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -46,6 +46,7 @@ class ShutdownTask implements ITask {
|
|||
private final boolean cleanupLeasesOfCompletedShards;
|
||||
private final TaskType taskType = TaskType.SHUTDOWN;
|
||||
private final long backoffTimeMillis;
|
||||
private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
|
|
@ -59,7 +60,8 @@ class ShutdownTask implements ITask {
|
|||
InitialPositionInStreamExtended initialPositionInStream,
|
||||
boolean cleanupLeasesOfCompletedShards,
|
||||
ILeaseManager<KinesisClientLease> leaseManager,
|
||||
long backoffTimeMillis) {
|
||||
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;
|
||||
|
|
|
|||
|
|
@ -0,0 +1,45 @@
|
|||
/*
|
||||
* 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;
|
||||
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).accept();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
//
|
||||
// Does nothing as this retriever doesn't manage any resources
|
||||
//
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isShutdown() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
@ -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<Integer> retryGetRecordsInSeconds;
|
||||
private final Optional<Integer> 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<Integer> retryGetRecordsInSeconds, Optional<Integer> 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());
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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<String> parentShardIds = new HashSet<String>();
|
||||
|
||||
|
|
@ -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<String> parentShardIds) {
|
||||
Long lastCounterIncrementNanos, ExtendedSequenceNumber checkpoint, ExtendedSequenceNumber pendingCheckpoint,
|
||||
Long ownerSwitchesSinceCheckpoint, Set<String> 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;
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
@ -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<KinesisCli
|
|||
private static final String OWNER_SWITCHES_KEY = "ownerSwitchesSinceCheckpoint";
|
||||
private static final String CHECKPOINT_SEQUENCE_NUMBER_KEY = "checkpoint";
|
||||
private static final String CHECKPOINT_SUBSEQUENCE_NUMBER_KEY = "checkpointSubSequenceNumber";
|
||||
private static final String PENDING_CHECKPOINT_SEQUENCE_KEY = "pendingCheckpoint";
|
||||
private static final String PENDING_CHECKPOINT_SUBSEQUENCE_KEY = "pendingCheckpointSubSequenceNumber";
|
||||
public final String PARENT_SHARD_ID_KEY = "parentShardId";
|
||||
|
||||
private final LeaseSerializer baseSerializer = new LeaseSerializer(KinesisClientLease.class);
|
||||
|
|
@ -50,6 +53,11 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer<KinesisCli
|
|||
result.put(PARENT_SHARD_ID_KEY, DynamoUtils.createAttributeValue(lease.getParentShardIds()));
|
||||
}
|
||||
|
||||
if (lease.getPendingCheckpoint() != null && !lease.getPendingCheckpoint().getSequenceNumber().isEmpty()) {
|
||||
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.getPendingCheckpoint().getSequenceNumber()));
|
||||
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, DynamoUtils.createAttributeValue(lease.getPendingCheckpoint().getSubSequenceNumber()));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
|
@ -65,6 +73,14 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer<KinesisCli
|
|||
);
|
||||
result.setParentShardIds(DynamoUtils.safeGetSS(dynamoRecord, PARENT_SHARD_ID_KEY));
|
||||
|
||||
if (!Strings.isNullOrEmpty(DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY))) {
|
||||
result.setPendingCheckpoint(
|
||||
new ExtendedSequenceNumber(
|
||||
DynamoUtils.safeGetString(dynamoRecord, PENDING_CHECKPOINT_SEQUENCE_KEY),
|
||||
DynamoUtils.safeGetLong(dynamoRecord, PENDING_CHECKPOINT_SUBSEQUENCE_KEY))
|
||||
);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
|
@ -128,6 +144,14 @@ public class KinesisClientLeaseSerializer implements ILeaseSerializer<KinesisCli
|
|||
new AttributeValueUpdate(DynamoUtils.createAttributeValue(lease.getOwnerSwitchesSinceCheckpoint()),
|
||||
AttributeAction.PUT));
|
||||
|
||||
if (lease.getPendingCheckpoint() != null && !lease.getPendingCheckpoint().getSequenceNumber().isEmpty()) {
|
||||
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue(lease.getPendingCheckpoint().getSequenceNumber()), AttributeAction.PUT));
|
||||
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, new AttributeValueUpdate(DynamoUtils.createAttributeValue(lease.getPendingCheckpoint().getSubSequenceNumber()), AttributeAction.PUT));
|
||||
} else {
|
||||
result.put(PENDING_CHECKPOINT_SEQUENCE_KEY, new AttributeValueUpdate().withAction(AttributeAction.DELETE));
|
||||
result.put(PENDING_CHECKPOINT_SUBSEQUENCE_KEY, new AttributeValueUpdate().withAction(AttributeAction.DELETE));
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
@ -577,6 +577,7 @@ public class LeaseManager<T extends Lease> implements ILeaseManager<T> {
|
|||
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
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -148,13 +148,14 @@ public class MultiLangDaemon implements Callable<Integer> {
|
|||
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<Void> 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);
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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[] {
|
||||
|
|
|
|||
|
|
@ -107,6 +107,68 @@ public abstract class CheckpointImplTestBase {
|
|||
ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(checkpointValue);
|
||||
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), concurrencyToken);
|
||||
Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpoint(shardId));
|
||||
Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint());
|
||||
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint());
|
||||
}
|
||||
|
||||
@Test
|
||||
public final void testInitialPrepareCheckpoint() throws Exception {
|
||||
String sequenceNumber = "1";
|
||||
String pendingCheckpointValue = "99999";
|
||||
String shardId = "myShardId";
|
||||
ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(sequenceNumber);
|
||||
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken);
|
||||
|
||||
ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue);
|
||||
checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), testConcurrencyToken);
|
||||
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint());
|
||||
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint());
|
||||
}
|
||||
|
||||
@Test
|
||||
public final void testAdvancingPrepareCheckpoint() throws Exception {
|
||||
String shardId = "myShardId";
|
||||
String checkpointValue = "12345";
|
||||
ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue);
|
||||
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), testConcurrencyToken);
|
||||
|
||||
for (Integer i = 0; i < 10; i++) {
|
||||
String sequenceNumber = i.toString();
|
||||
ExtendedSequenceNumber extendedSequenceNumber = new ExtendedSequenceNumber(sequenceNumber);
|
||||
checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(sequenceNumber), testConcurrencyToken);
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint());
|
||||
Assert.assertEquals(extendedSequenceNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public final void testPrepareAndSetCheckpoint() throws Exception {
|
||||
String checkpointValue = "12345";
|
||||
String shardId = "testShardId-1";
|
||||
String concurrencyToken = "token-1";
|
||||
String pendingCheckpointValue = "99999";
|
||||
|
||||
// set initial checkpoint
|
||||
ExtendedSequenceNumber extendedCheckpointNumber = new ExtendedSequenceNumber(checkpointValue);
|
||||
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(checkpointValue), concurrencyToken);
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint());
|
||||
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint());
|
||||
|
||||
// prepare checkpoint
|
||||
ExtendedSequenceNumber extendedPendingCheckpointNumber = new ExtendedSequenceNumber(pendingCheckpointValue);
|
||||
checkpoint.prepareCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken);
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||
Assert.assertEquals(extendedCheckpointNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint());
|
||||
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint());
|
||||
|
||||
// do checkpoint
|
||||
checkpoint.setCheckpoint(shardId, new ExtendedSequenceNumber(pendingCheckpointValue), concurrencyToken);
|
||||
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpoint(shardId));
|
||||
Assert.assertEquals(extendedPendingCheckpointNumber, checkpoint.getCheckpointObject(shardId).getCheckpoint());
|
||||
Assert.assertEquals(null, checkpoint.getCheckpointObject(shardId).getPendingCheckpoint());
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -33,6 +33,7 @@ public class InMemoryCheckpointImpl implements ICheckpoint {
|
|||
|
||||
private Map<String, ExtendedSequenceNumber> checkpoints = new HashMap<>();
|
||||
private Map<String, ExtendedSequenceNumber> flushpoints = new HashMap<>();
|
||||
private Map<String, ExtendedSequenceNumber> 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) {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,171 @@
|
|||
/*
|
||||
* 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.ExecutorCompletionService;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.RejectedExecutionHandler;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
|
||||
import static org.hamcrest.CoreMatchers.nullValue;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.core.IsEqual.equalTo;
|
||||
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;
|
||||
@Mock
|
||||
private Supplier<CompletionService<DataFetcherResult>> completionServiceSupplier;
|
||||
@Mock
|
||||
private DataFetcherResult result;
|
||||
@Mock
|
||||
private GetRecordsResult recordsResult;
|
||||
|
||||
private CompletionService<DataFetcherResult> completionService;
|
||||
|
||||
private AsynchronousGetRecordsRetrievalStrategy getRecordsRetrivalStrategy;
|
||||
private KinesisDataFetcher dataFetcher;
|
||||
private ExecutorService executorService;
|
||||
private RejectedExecutionHandler rejectedExecutionHandler;
|
||||
private int numberOfRecords = 10;
|
||||
|
||||
|
||||
@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));
|
||||
completionService = spy(new ExecutorCompletionService<DataFetcherResult>(executorService));
|
||||
when(completionServiceSupplier.get()).thenReturn(completionService);
|
||||
getRecordsRetrivalStrategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher, executorService, RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, "shardId-0001");
|
||||
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());
|
||||
assertNull(getRecordsResult);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void multiRequestTest() {
|
||||
ExecutorCompletionService<DataFetcherResult> completionService1 = spy(new ExecutorCompletionService<DataFetcherResult>(executorService));
|
||||
when(completionServiceSupplier.get()).thenReturn(completionService1);
|
||||
GetRecordsResult getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords);
|
||||
verify(dataFetcher, atLeast(getLeastNumberOfCalls())).getRecords(numberOfRecords);
|
||||
verify(executorService, atLeast(getLeastNumberOfCalls())).execute(any());
|
||||
assertThat(getRecordsResult, equalTo(recordsResult));
|
||||
|
||||
when(result.accept()).thenReturn(null);
|
||||
ExecutorCompletionService<DataFetcherResult> completionService2 = spy(new ExecutorCompletionService<DataFetcherResult>(executorService));
|
||||
when(completionServiceSupplier.get()).thenReturn(completionService2);
|
||||
getRecordsResult = getRecordsRetrivalStrategy.getRecords(numberOfRecords);
|
||||
assertThat(getRecordsResult, nullValue(GetRecordsResult.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore
|
||||
public void testInterrupted() throws InterruptedException, ExecutionException {
|
||||
|
||||
Future<DataFetcherResult> 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 DataFetcherResult getRecords(final int maxRecords) {
|
||||
try {
|
||||
Thread.sleep(SLEEP_GET_RECORDS_IN_SECONDS * 1000);
|
||||
} catch (InterruptedException e) {
|
||||
// Do nothing
|
||||
}
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,157 @@
|
|||
/*
|
||||
* 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;
|
||||
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 java.util.function.Supplier;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.runners.MockitoJUnitRunner;
|
||||
|
||||
import com.amazonaws.services.kinesis.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 Supplier<CompletionService<DataFetcherResult>> completionServiceSupplier;
|
||||
@Mock
|
||||
private CompletionService<DataFetcherResult> completionService;
|
||||
@Mock
|
||||
private Future<DataFetcherResult> successfulFuture;
|
||||
@Mock
|
||||
private Future<DataFetcherResult> 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
|
||||
public void testSingleSuccessfulRequestFuture() throws Exception {
|
||||
AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher,
|
||||
executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, SHARD_ID);
|
||||
|
||||
when(executorService.isShutdown()).thenReturn(false);
|
||||
when(completionService.submit(any())).thenReturn(successfulFuture);
|
||||
when(completionService.poll(anyLong(), any())).thenReturn(successfulFuture);
|
||||
when(successfulFuture.get()).thenReturn(dataFetcherResult);
|
||||
|
||||
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));
|
||||
|
||||
assertThat(result, equalTo(expectedResults));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBlockedAndSuccessfulFuture() throws Exception {
|
||||
AsynchronousGetRecordsRetrievalStrategy strategy = new AsynchronousGetRecordsRetrievalStrategy(dataFetcher,
|
||||
executorService, (int) RETRY_GET_RECORDS_IN_SECONDS, completionServiceSupplier, 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(dataFetcherResult);
|
||||
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));
|
||||
|
||||
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, completionServiceSupplier, 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, completionServiceSupplier, 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(dataFetcherResult);
|
||||
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));
|
||||
|
||||
|
||||
assertThat(actualResult, equalTo(expectedResults));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -74,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;
|
||||
|
|
@ -96,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<ILeaseManager<KinesisClientLease>> LEASE_MANAGER_CLASS = (Class<ILeaseManager<KinesisClientLease>>) (Class<?>) ILeaseManager.class;
|
||||
|
|
@ -152,7 +154,34 @@ public class ConsumerStatesTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void processingStateTest() {
|
||||
public void processingStateTestSynchronous() {
|
||||
ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState();
|
||||
ITask task = state.createTask(consumer);
|
||||
|
||||
assertThat(task, procTask(ShardInfo.class, "shardInfo", equalTo(shardInfo)));
|
||||
assertThat(task, procTask(IRecordProcessor.class, "recordProcessor", equalTo(recordProcessor)));
|
||||
assertThat(task, procTask(RecordProcessorCheckpointer.class, "recordProcessorCheckpointer",
|
||||
equalTo(recordProcessorCheckpointer)));
|
||||
assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher)));
|
||||
assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig)));
|
||||
assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis)));
|
||||
|
||||
assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState()));
|
||||
|
||||
assertThat(state.shutdownTransition(ShutdownReason.ZOMBIE),
|
||||
equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState()));
|
||||
assertThat(state.shutdownTransition(ShutdownReason.TERMINATE),
|
||||
equalTo(ShardConsumerState.SHUTTING_DOWN.getConsumerState()));
|
||||
assertThat(state.shutdownTransition(ShutdownReason.REQUESTED),
|
||||
equalTo(ShardConsumerState.SHUTDOWN_REQUESTED.getConsumerState()));
|
||||
|
||||
assertThat(state.getState(), equalTo(ShardConsumerState.PROCESSING));
|
||||
assertThat(state.getTaskType(), equalTo(TaskType.PROCESS));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void processingStateTestAsynchronous() {
|
||||
ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState();
|
||||
ITask task = state.createTask(consumer);
|
||||
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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());
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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
|
||||
|
|
@ -117,6 +136,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 +158,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,13 +201,90 @@ 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());
|
||||
}
|
||||
|
||||
@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 {
|
||||
|
|
@ -206,8 +303,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<Record> actualRecords = fetcher.getRecords(MAX_RECORDS).getRecords();
|
||||
List<Record> actualRecords = getRecordsRetrievalStrategy.getRecords(MAX_RECORDS).getRecords();
|
||||
|
||||
Assert.assertEquals(expectedRecords, actualRecords);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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<Record> 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<Record> 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<ProcessRecordsInput> priCaptor = ArgumentCaptor.forClass(ProcessRecordsInput.class);
|
||||
verify(mockRecordProcessor).processRecords(priCaptor.capture());
|
||||
processedRecords = priCaptor.getValue().getRecords();
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
|
||||
for (LinkedHashMap<String, CheckpointAction> testPlan : getMixedCallsTestPlan()) {
|
||||
RecordProcessorCheckpointer processingCheckpointer =
|
||||
new RecordProcessorCheckpointer(shardInfo, checkpoint, validator);
|
||||
testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.CHECKPOINTER);
|
||||
}
|
||||
}
|
||||
|
||||
List<LinkedHashMap<String, CheckpointAction>> testPlans =
|
||||
new ArrayList<LinkedHashMap<String, CheckpointAction>>();
|
||||
/**
|
||||
* 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<String, CheckpointAction> 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<String, CheckpointAction> testPlan : getMixedCallsTestPlan()) {
|
||||
RecordProcessorCheckpointer processingCheckpointer =
|
||||
new RecordProcessorCheckpointer(shardInfo, checkpoint, validator);
|
||||
testMixedCheckpointCalls(processingCheckpointer, testPlan, CheckpointerType.PREPARE_THEN_CHECKPOINTER);
|
||||
}
|
||||
}
|
||||
|
||||
private List<LinkedHashMap<String, CheckpointAction>> getMixedCallsTestPlan() {
|
||||
List<LinkedHashMap<String, CheckpointAction>> testPlans = new ArrayList<LinkedHashMap<String, CheckpointAction>>();
|
||||
|
||||
/*
|
||||
* Simulate a scenario where the checkpointer is created at "latest".
|
||||
|
|
@ -356,11 +699,7 @@ public class RecordProcessorCheckpointerTest {
|
|||
}
|
||||
});
|
||||
|
||||
for (LinkedHashMap<String, CheckpointAction> 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<String, CheckpointAction> checkpointValueAndAction) throws Exception {
|
||||
LinkedHashMap<String, CheckpointAction> checkpointValueAndAction,
|
||||
CheckpointerType checkpointerType) throws Exception {
|
||||
|
||||
for (Entry<String, CheckpointAction> entry : checkpointValueAndAction.entrySet()) {
|
||||
IPreparedCheckpointer preparedCheckpoint = null;
|
||||
ExtendedSequenceNumber lastCheckpointValue = processingCheckpointer.getLastCheckpointValue();
|
||||
|
||||
if (SentinelCheckpoint.SHARD_END.toString().equals(entry.getKey())) {
|
||||
|
|
@ -400,11 +741,35 @@ public class RecordProcessorCheckpointerTest {
|
|||
processingCheckpointer.getLastCheckpointValue());
|
||||
continue;
|
||||
case NO_SEQUENCE_NUMBER:
|
||||
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:
|
||||
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
|
||||
Assert.assertEquals("Expected the last checkpoint value to change after checkpointing",
|
||||
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -20,9 +20,11 @@ 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;
|
||||
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 +39,8 @@ import java.util.ArrayList;
|
|||
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;
|
||||
|
|
@ -45,6 +49,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 +59,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 +116,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 +165,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 +228,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 +246,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 +258,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 +464,110 @@ 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)));
|
||||
}
|
||||
|
||||
@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<Record> expectedRecords,
|
||||
List<Record> actualRecords) {
|
||||
|
|
@ -469,4 +590,21 @@ public class ShardConsumerTest {
|
|||
}
|
||||
return userRecords;
|
||||
}
|
||||
|
||||
Matcher<InitializationInput> initializationInputMatcher(final ExtendedSequenceNumber checkpoint,
|
||||
final ExtendedSequenceNumber pendingCheckpoint) {
|
||||
return new TypeSafeMatcher<InitializationInput>() {
|
||||
@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));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -14,7 +14,9 @@
|
|||
*/
|
||||
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 =
|
||||
|
|
@ -52,6 +58,9 @@ public class ShutdownTaskTest {
|
|||
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());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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<String> 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);
|
||||
}
|
||||
}
|
||||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -1,5 +1,5 @@
|
|||
/*
|
||||
* 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.
|
||||
|
|
|
|||
Loading…
Reference in a new issue