merged chagnes
This commit is contained in:
commit
f841ea2cd9
11 changed files with 182 additions and 127 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.3
|
||||
Bundle-Vendor: Amazon Technologies, Inc
|
||||
Bundle-RequiredExecutionEnvironment: JavaSE-1.7
|
||||
Require-Bundle: org.apache.commons.codec;bundle-version="1.6",
|
||||
|
|
|
|||
10
README.md
10
README.md
|
|
@ -29,6 +29,12 @@ For producer-side developers using the **[Kinesis Producer Library (KPL)][kinesi
|
|||
To make it easier for developers to write record processors in other languages, we have implemented a Java based daemon, called MultiLangDaemon that does all the heavy lifting. Our approach has the daemon spawn a sub-process, which in turn runs the record processor, which can be written in any language. The MultiLangDaemon process and the record processor sub-process communicate with each other over [STDIN and STDOUT using a defined protocol][multi-lang-protocol]. There will be a one to one correspondence amongst record processors, child processes, and shards. For Python developers specifically, we have abstracted these implementation details away and [expose an interface][kclpy] that enables you to focus on writing record processing logic in Python. This approach enables KCL to be language agnostic, while providing identical features and similar parallel processing model across all languages.
|
||||
|
||||
## Release Notes
|
||||
### Release 1.8.3 (September 22, 2017)
|
||||
* Call shutdown on the retriever when the record processor is being shutdown
|
||||
This fixes a bug that could leak threads if using the [`AsynchronousGetRecordsRetrievalStrategy`](https://github.com/awslabs/amazon-kinesis-client/blob/9a82b6bd05b3c9c5f8581af007141fa6d5f0fc4e/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/AsynchronousGetRecordsRetrievalStrategy.java#L42) is being used.
|
||||
The asynchronous retriever is only used when [`KinesisClientLibConfiguration#retryGetRecordsInSeconds`](https://github.com/awslabs/amazon-kinesis-client/blob/01d2688bc6e68fd3fe5cb698cb65299d67ac930d/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L227), and [`KinesisClientLibConfiguration#maxGetRecordsThreadPool`](https://github.com/awslabs/amazon-kinesis-client/blob/01d2688bc6e68fd3fe5cb698cb65299d67ac930d/src/main/java/com/amazonaws/services/kinesis/clientlibrary/lib/worker/KinesisClientLibConfiguration.java#L230) are set.
|
||||
* [PR #222](https://github.com/awslabs/amazon-kinesis-client/pull/222)
|
||||
|
||||
### Release 1.8.2 (September 20, 2017)
|
||||
* Add support for two phase checkpoints
|
||||
Applications can now set a pending checkpoint, before completing the checkpoint operation. Once the application has completed its checkpoint steps, the final checkpoint will clear the pending checkpoint.
|
||||
|
|
@ -36,10 +42,10 @@ To make it easier for developers to write record processors in other languages,
|
|||
* [PR #188](https://github.com/awslabs/amazon-kinesis-client/pull/188)
|
||||
* Support timeouts, and retry for GetRecords calls.
|
||||
Applications can now set timeouts for GetRecord calls to Kinesis. As part of setting the timeout, the application must also provide a thread pool size for concurrent requests.
|
||||
* [PR #214](https://github.com/awslabs/amazon-kinesis-client/pulls/214)
|
||||
* [PR #214](https://github.com/awslabs/amazon-kinesis-client/pull/214)
|
||||
* Notification when the lease table is throttled
|
||||
When writes, or reads, to the lease table are throttled a warning will be emitted. If you're seeing this warning you should increase the IOPs for your lease table to prevent processing delays.
|
||||
* [PR #212](https://github.com/awslabs/amazon-kinesis-client/pulls/212)
|
||||
* [PR #212](https://github.com/awslabs/amazon-kinesis-client/pull/212)
|
||||
* Support configuring the graceful shutdown timeout for MultiLang Clients
|
||||
This adds support for setting the timeout that the Java process will wait for the MutliLang client to complete graceful shutdown. The timeout can be configured by adding `shutdownGraceMillis` to the properties file set to the number of milliseconds to wait.
|
||||
* [PR #204](https://github.com/awslabs/amazon-kinesis-client/pull/204)
|
||||
|
|
|
|||
2
pom.xml
2
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</version>
|
||||
<version>1.8.3</version>
|
||||
<description>The Amazon Kinesis Client Library for Java enables Java developers to easily consume and process data
|
||||
from Amazon Kinesis.
|
||||
</description>
|
||||
|
|
|
|||
|
|
@ -308,10 +308,9 @@ class ConsumerStates {
|
|||
@Override
|
||||
public ITask createTask(ShardConsumer consumer) {
|
||||
return new ProcessTask(consumer.getShardInfo(), consumer.getStreamConfig(), consumer.getRecordProcessor(),
|
||||
consumer.getRecordProcessorCheckpointer(),
|
||||
consumer.getDataFetcher(), consumer.getTaskBackoffTimeMillis(),
|
||||
consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist(),
|
||||
consumer.getGetRecordsCache());
|
||||
consumer.getRecordProcessorCheckpointer(), consumer.getDataFetcher(),
|
||||
consumer.getTaskBackoffTimeMillis(), consumer.isSkipShardSyncAtWorkerInitializationIfLeasesExist(),
|
||||
consumer.getGetRecordsRetrievalStrategy());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
@ -515,7 +514,8 @@ class ConsumerStates {
|
|||
consumer.getStreamConfig().getStreamProxy(),
|
||||
consumer.getStreamConfig().getInitialPositionInStream(),
|
||||
consumer.isCleanupLeasesOfCompletedShards(), consumer.getLeaseManager(),
|
||||
consumer.getTaskBackoffTimeMillis());
|
||||
consumer.getTaskBackoffTimeMillis(),
|
||||
consumer.getGetRecordsRetrievalStrategy());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -126,7 +126,7 @@ public class KinesisClientLibConfiguration {
|
|||
/**
|
||||
* User agent set when Amazon Kinesis Client Library makes AWS requests.
|
||||
*/
|
||||
public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.2";
|
||||
public static final String KINESIS_CLIENT_LIB_USER_AGENT = "amazon-kinesis-client-library-java-1.8.3";
|
||||
|
||||
/**
|
||||
* KCL will validate client provided sequence numbers with a call to Amazon Kinesis before checkpointing for calls
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@
|
|||
package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
|
||||
|
||||
import java.math.BigInteger;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.ListIterator;
|
||||
import java.util.Optional;
|
||||
|
|
@ -33,6 +34,7 @@ import com.amazonaws.services.kinesis.metrics.impl.MetricsHelper;
|
|||
import com.amazonaws.services.kinesis.metrics.interfaces.IMetricsScope;
|
||||
import com.amazonaws.services.kinesis.metrics.interfaces.MetricsLevel;
|
||||
import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
|
||||
import com.amazonaws.services.kinesis.model.GetRecordsResult;
|
||||
import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException;
|
||||
import com.amazonaws.services.kinesis.model.Record;
|
||||
import com.amazonaws.services.kinesis.model.Shard;
|
||||
|
|
@ -53,7 +55,6 @@ class ProcessTask implements ITask {
|
|||
|
||||
private final ShardInfo shardInfo;
|
||||
private final IRecordProcessor recordProcessor;
|
||||
private final GetRecordsCache getRecordsCache;
|
||||
private final RecordProcessorCheckpointer recordProcessorCheckpointer;
|
||||
private final KinesisDataFetcher dataFetcher;
|
||||
private final TaskType taskType = TaskType.PROCESS;
|
||||
|
|
@ -62,16 +63,7 @@ class ProcessTask implements ITask {
|
|||
private final Shard shard;
|
||||
private final ThrottlingReporter throttlingReporter;
|
||||
|
||||
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));
|
||||
}
|
||||
private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
||||
/**
|
||||
* @param shardInfo
|
||||
|
|
@ -86,17 +78,17 @@ class ProcessTask implements ITask {
|
|||
* Kinesis data fetcher (used to fetch records from Kinesis)
|
||||
* @param backoffTimeMillis
|
||||
* backoff time when catching exceptions
|
||||
* @param getRecordsCache
|
||||
* Record processor factory to create recordFetcher object
|
||||
* @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,
|
||||
GetRecordsCache getRecordsCache) {
|
||||
this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher,
|
||||
backoffTimeMillis, skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()), getRecordsCache);
|
||||
RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher,
|
||||
long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) {
|
||||
this(shardInfo, streamConfig, recordProcessor, recordProcessorCheckpointer, dataFetcher, backoffTimeMillis,
|
||||
skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
new ThrottlingReporter(MAX_CONSECUTIVE_THROTTLES, shardInfo.getShardId()),
|
||||
getRecordsRetrievalStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -106,8 +98,6 @@ class ProcessTask implements ITask {
|
|||
* Stream configuration
|
||||
* @param recordProcessor
|
||||
* Record processor used to process the data records for the shard
|
||||
* @param getRecordsCache
|
||||
* RecordFetcher factory used to create recordFetcher object
|
||||
* @param recordProcessorCheckpointer
|
||||
* Passed to the RecordProcessor so it can checkpoint progress
|
||||
* @param dataFetcher
|
||||
|
|
@ -118,9 +108,9 @@ class ProcessTask implements ITask {
|
|||
* determines how throttling events should be reported in the log.
|
||||
*/
|
||||
public ProcessTask(ShardInfo shardInfo, StreamConfig streamConfig, IRecordProcessor recordProcessor,
|
||||
RecordProcessorCheckpointer recordProcessorCheckpointer,
|
||||
KinesisDataFetcher dataFetcher, long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
ThrottlingReporter throttlingReporter, GetRecordsCache getRecordsCache) {
|
||||
RecordProcessorCheckpointer recordProcessorCheckpointer, KinesisDataFetcher dataFetcher,
|
||||
long backoffTimeMillis, boolean skipShardSyncAtWorkerInitializationIfLeasesExist,
|
||||
ThrottlingReporter throttlingReporter, GetRecordsRetrievalStrategy getRecordsRetrievalStrategy) {
|
||||
super();
|
||||
this.shardInfo = shardInfo;
|
||||
this.recordProcessor = recordProcessor;
|
||||
|
|
@ -130,7 +120,7 @@ class ProcessTask implements ITask {
|
|||
this.backoffTimeMillis = backoffTimeMillis;
|
||||
this.throttlingReporter = throttlingReporter;
|
||||
IKinesisProxy kinesisProxy = this.streamConfig.getStreamProxy();
|
||||
this.getRecordsCache = getRecordsCache;
|
||||
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
|
||||
|
|
@ -168,9 +158,9 @@ class ProcessTask implements ITask {
|
|||
return new TaskResult(null, true);
|
||||
}
|
||||
|
||||
final ProcessRecordsInput processRecordsInput = getRecordsResult();
|
||||
final GetRecordsResult getRecordsResult = getRecordsResult();
|
||||
throttlingReporter.success();
|
||||
List<Record> records = processRecordsInput.getRecords();
|
||||
List<Record> records = getRecordsResult.getRecords();
|
||||
|
||||
if (!records.isEmpty()) {
|
||||
scope.addData(RECORDS_PROCESSED_METRIC, records.size(), StandardUnit.Count, MetricsLevel.SUMMARY);
|
||||
|
|
@ -185,7 +175,7 @@ class ProcessTask implements ITask {
|
|||
recordProcessorCheckpointer.getLargestPermittedCheckpointValue()));
|
||||
|
||||
if (shouldCallProcessRecords(records)) {
|
||||
callProcessRecords(processRecordsInput);
|
||||
callProcessRecords(getRecordsResult, records);
|
||||
}
|
||||
} catch (ProvisionedThroughputExceededException pte) {
|
||||
throttlingReporter.throttled();
|
||||
|
|
@ -216,14 +206,17 @@ class ProcessTask implements ITask {
|
|||
/**
|
||||
* Dispatches a batch of records to the record processor, and handles any fallout from that.
|
||||
*
|
||||
* @param processRecordsInput
|
||||
* the ProcessRecordsInput result of the last call to Kinesis
|
||||
* @param getRecordsResult
|
||||
* the result of the last call to Kinesis
|
||||
* @param records
|
||||
* the records to be dispatched. It's possible the records have been adjusted by KPL deaggregation.
|
||||
*/
|
||||
private void callProcessRecords(ProcessRecordsInput processRecordsInput) {
|
||||
List<Record> records = processRecordsInput.getRecords();
|
||||
private void callProcessRecords(GetRecordsResult getRecordsResult, List<Record> records) {
|
||||
LOG.debug("Calling application processRecords() with " + records.size() + " records from "
|
||||
+ shardInfo.getShardId());
|
||||
processRecordsInput.withCheckpointer(recordProcessorCheckpointer);
|
||||
final ProcessRecordsInput processRecordsInput = new ProcessRecordsInput().withRecords(records)
|
||||
.withCheckpointer(recordProcessorCheckpointer)
|
||||
.withMillisBehindLatest(getRecordsResult.getMillisBehindLatest());
|
||||
|
||||
final long recordProcessorStartTimeMillis = System.currentTimeMillis();
|
||||
try {
|
||||
|
|
@ -346,7 +339,7 @@ class ProcessTask implements ITask {
|
|||
*
|
||||
* @return list of data records from Kinesis
|
||||
*/
|
||||
private ProcessRecordsInput getRecordsResult() {
|
||||
private GetRecordsResult getRecordsResult() {
|
||||
try {
|
||||
return getRecordsResultAndRecordMillisBehindLatest();
|
||||
} catch (ExpiredIteratorException e) {
|
||||
|
|
@ -382,17 +375,22 @@ class ProcessTask implements ITask {
|
|||
*
|
||||
* @return list of data records from Kinesis
|
||||
*/
|
||||
private ProcessRecordsInput getRecordsResultAndRecordMillisBehindLatest() {
|
||||
final ProcessRecordsInput processRecordsInput = getRecordsCache.getNextResult();
|
||||
private GetRecordsResult getRecordsResultAndRecordMillisBehindLatest() {
|
||||
final GetRecordsResult getRecordsResult = getRecordsRetrievalStrategy.getRecords(streamConfig.getMaxRecords());
|
||||
|
||||
if (processRecordsInput.getMillisBehindLatest() != null) {
|
||||
if (getRecordsResult == null) {
|
||||
// Stream no longer exists
|
||||
return new GetRecordsResult().withRecords(Collections.<Record>emptyList());
|
||||
}
|
||||
|
||||
if (getRecordsResult.getMillisBehindLatest() != null) {
|
||||
MetricsHelper.getMetricsScope().addData(MILLIS_BEHIND_LATEST_METRIC,
|
||||
processRecordsInput.getMillisBehindLatest(),
|
||||
getRecordsResult.getMillisBehindLatest(),
|
||||
StandardUnit.Milliseconds,
|
||||
MetricsLevel.SUMMARY);
|
||||
}
|
||||
|
||||
return processRecordsInput;
|
||||
return getRecordsResult;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -56,16 +56,13 @@ class ShardConsumer {
|
|||
private final boolean cleanupLeasesOfCompletedShards;
|
||||
private final long taskBackoffTimeMillis;
|
||||
private final boolean skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
@Getter
|
||||
private final Optional<Integer> retryGetRecordsInSeconds;
|
||||
@Getter
|
||||
private final Optional<Integer> maxGetRecordsThreadPool;
|
||||
|
||||
private ITask currentTask;
|
||||
private long currentTaskSubmitTime;
|
||||
private Future<TaskResult> future;
|
||||
@Getter
|
||||
private final GetRecordsCache getRecordsCache;
|
||||
private final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
||||
|
||||
private static final GetRecordsRetrievalStrategy makeStrategy(KinesisDataFetcher dataFetcher,
|
||||
Optional<Integer> retryGetRecordsInSeconds,
|
||||
|
|
@ -168,11 +165,7 @@ class ShardConsumer {
|
|||
this.cleanupLeasesOfCompletedShards = cleanupLeasesOfCompletedShards;
|
||||
this.taskBackoffTimeMillis = backoffTimeMillis;
|
||||
this.skipShardSyncAtWorkerInitializationIfLeasesExist = skipShardSyncAtWorkerInitializationIfLeasesExist;
|
||||
this.retryGetRecordsInSeconds = retryGetRecordsInSeconds;
|
||||
this.maxGetRecordsThreadPool = maxGetRecordsThreadPool;
|
||||
this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher(
|
||||
makeStrategy(this.dataFetcher, this.retryGetRecordsInSeconds,
|
||||
this.maxGetRecordsThreadPool, this.shardInfo));
|
||||
this.getRecordsRetrievalStrategy = makeStrategy(dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, shardInfo);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -17,17 +17,14 @@ package com.amazonaws.services.kinesis.clientlibrary.lib.worker;
|
|||
import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerStates.ConsumerState;
|
||||
import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.ConsumerStates.ShardConsumerState;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.instanceOf;
|
||||
import static org.hamcrest.CoreMatchers.nullValue;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.mockito.Matchers.any;
|
||||
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.lang.reflect.Field;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
|
|
@ -80,9 +77,7 @@ public class ConsumerStatesTest {
|
|||
@Mock
|
||||
private InitialPositionInStreamExtended initialPositionInStream;
|
||||
@Mock
|
||||
private RecordsFetcherFactory recordsFetcherFactory;
|
||||
@Mock
|
||||
private GetRecordsCache recordsFetcher;
|
||||
private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
|
||||
|
||||
private long parentShardPollIntervalMillis = 0xCAFE;
|
||||
private boolean cleanupLeasesOfCompletedShards = true;
|
||||
|
|
@ -105,8 +100,7 @@ public class ConsumerStatesTest {
|
|||
when(consumer.isCleanupLeasesOfCompletedShards()).thenReturn(cleanupLeasesOfCompletedShards);
|
||||
when(consumer.getTaskBackoffTimeMillis()).thenReturn(taskBackoffTimeMillis);
|
||||
when(consumer.getShutdownReason()).thenReturn(reason);
|
||||
when(consumer.getConfig()).thenReturn(config);
|
||||
when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory);
|
||||
when(consumer.getGetRecordsRetrievalStrategy()).thenReturn(getRecordsRetrievalStrategy);
|
||||
}
|
||||
|
||||
private static final Class<ILeaseManager<KinesisClientLease>> LEASE_MANAGER_CLASS = (Class<ILeaseManager<KinesisClientLease>>) (Class<?>) ILeaseManager.class;
|
||||
|
|
@ -163,9 +157,6 @@ public class ConsumerStatesTest {
|
|||
|
||||
@Test
|
||||
public void processingStateTestSynchronous() {
|
||||
when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.empty());
|
||||
when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.empty());
|
||||
|
||||
ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState();
|
||||
ITask task = state.createTask(consumer);
|
||||
|
||||
|
|
@ -176,7 +167,6 @@ public class ConsumerStatesTest {
|
|||
assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher)));
|
||||
assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig)));
|
||||
assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis)));
|
||||
assertThat(task, procTask(GetRecordsRetrievalStrategy.class, "getRecordsRetrievalStrategy", instanceOf(SynchronousGetRecordsRetrievalStrategy.class) ));
|
||||
|
||||
assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState()));
|
||||
|
||||
|
|
@ -194,9 +184,6 @@ public class ConsumerStatesTest {
|
|||
|
||||
@Test
|
||||
public void processingStateTestAsynchronous() {
|
||||
when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.of(1));
|
||||
when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.of(2));
|
||||
|
||||
ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState();
|
||||
ITask task = state.createTask(consumer);
|
||||
|
||||
|
|
@ -207,7 +194,6 @@ public class ConsumerStatesTest {
|
|||
assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher)));
|
||||
assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig)));
|
||||
assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis)));
|
||||
assertThat(task, procTask(GetRecordsRetrievalStrategy.class, "getRecordsRetrievalStrategy", instanceOf(AsynchronousGetRecordsRetrievalStrategy.class) ));
|
||||
|
||||
assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState()));
|
||||
|
||||
|
|
@ -225,9 +211,6 @@ public class ConsumerStatesTest {
|
|||
|
||||
@Test
|
||||
public void processingStateRecordsFetcher() {
|
||||
when(consumer.getMaxGetRecordsThreadPool()).thenReturn(Optional.of(1));
|
||||
when(consumer.getRetryGetRecordsInSeconds()).thenReturn(Optional.of(2));
|
||||
when(recordsFetcherFactory.createRecordsFetcher((any()))).thenReturn(recordsFetcher);
|
||||
|
||||
ConsumerState state = ShardConsumerState.PROCESSING.getConsumerState();
|
||||
ITask task = state.createTask(consumer);
|
||||
|
|
@ -239,7 +222,6 @@ public class ConsumerStatesTest {
|
|||
assertThat(task, procTask(KinesisDataFetcher.class, "dataFetcher", equalTo(dataFetcher)));
|
||||
assertThat(task, procTask(StreamConfig.class, "streamConfig", equalTo(streamConfig)));
|
||||
assertThat(task, procTask(Long.class, "backoffTimeMillis", equalTo(taskBackoffTimeMillis)));
|
||||
assertThat(task, procTask(GetRecordsCache.class, "recordsFetcher", equalTo(recordsFetcher)));
|
||||
|
||||
assertThat(state.successTransition(), equalTo(ShardConsumerState.PROCESSING.getConsumerState()));
|
||||
|
||||
|
|
|
|||
|
|
@ -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,6 +20,7 @@ import static org.hamcrest.Matchers.is;
|
|||
import static org.hamcrest.Matchers.nullValue;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyString;
|
||||
|
|
@ -40,6 +41,7 @@ import java.util.Date;
|
|||
import java.util.List;
|
||||
import java.util.ListIterator;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
|
@ -523,6 +525,62 @@ public class ShardConsumerTest {
|
|||
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) {
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
|||
Loading…
Reference in a new issue