From b1f1e6fba5ee0990e6c30ce6628f97811cb465bd Mon Sep 17 00:00:00 2001 From: Gordon Tai Date: Wed, 8 Jun 2016 18:46:02 +0800 Subject: [PATCH 1/2] [FLINK-3231] FlinkKinesisConsumer rework to handle Kinesis resharding This also fixes FLINK-4020: remove shard list querying from Kinesis consumer constructor --- docs/apis/streaming/connectors/kinesis.md | 13 +- docs/apis/streaming/fault_tolerance.md | 2 +- .../kinesis/FlinkKinesisConsumer.java | 253 +++----- .../config/KinesisConfigConstants.java | 67 +- .../kinesis/internals/KinesisDataFetcher.java | 570 +++++++++++++++--- ...ConsumerThread.java => ShardConsumer.java} | 138 +++-- .../kinesis/model/KinesisStreamShard.java | 88 ++- .../model/KinesisStreamShardState.java | 71 +++ .../kinesis/model/SentinelSequenceNumber.java | 4 - .../kinesis/proxy/GetShardListResult.java | 75 +++ .../kinesis/proxy/KinesisProxy.java | 296 ++++++--- .../kinesis/proxy/KinesisProxyInterface.java | 69 +++ .../connectors/kinesis/util/AWSUtil.java | 22 + .../kinesis/util/KinesisConfigUtil.java | 91 ++- .../kinesis/FlinkKinesisConsumerTest.java | 504 +++++++--------- .../internals/KinesisDataFetcherTest.java | 487 ++++++++++++++- .../kinesis/internals/ShardConsumerTest.java | 82 +++ .../internals/ShardConsumerThreadTest.java | 222 ------- .../manualtests/ManualExactlyOnceTest.java | 217 ++----- ...alExactlyOnceWithStreamReshardingTest.java | 247 ++++++++ .../ExactlyOnceValidatingConsumerThread.java | 156 +++++ .../FakeKinesisBehavioursFactory.java | 202 +++++++ .../KinesisEventsGeneratorProducerThread.java | 118 ++++ .../testutils/KinesisShardIdGenerator.java | 3 +- .../ReferenceKinesisShardTopologies.java | 117 ---- .../TestableFlinkKinesisConsumer.java | 54 +- .../testutils/TestableKinesisDataFetcher.java | 122 ++++ 27 files changed, 2947 insertions(+), 1343 deletions(-) rename flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/{ShardConsumerThread.java => ShardConsumer.java} (54%) create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java delete mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java delete mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java create mode 100644 flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java diff --git a/docs/apis/streaming/connectors/kinesis.md b/docs/apis/streaming/connectors/kinesis.md index db3a9c4c65d35..a539956867911 100644 --- a/docs/apis/streaming/connectors/kinesis.md +++ b/docs/apis/streaming/connectors/kinesis.md @@ -60,10 +60,10 @@ to setup Kinesis streams. Make sure to create the appropriate IAM policy and use ### Kinesis Consumer -The `FlinkKinesisConsumer` can be used to pull data from multiple Kinesis streams within the same AWS region in parallel. -It participates in Flink's distributed snapshot checkpointing and provides exactly-once user-defined state update guarantees. Note -that the current version can not handle resharding of Kinesis streams. When Kinesis streams are resharded, the consumer -will fail and the Flink streaming job must be resubmitted. +The `FlinkKinesisConsumer` is an exactly-once parallel streaming data source that subscribes to multiple AWS Kinesis +streams within the same AWS service region, and can handle resharding of streams. Each subtask of the consumer is +responsible for fetching data records from multiple Kinesis shards. The number of shards fetched by each subtask will +change as shards are closed and created by Kinesis. Before consuming data from Kinesis streams, make sure that all streams are created with the status "ACTIVE" in the AWS dashboard. @@ -114,6 +114,11 @@ the AWS access key ID and secret key are directly supplied in the configuration from the newest position in the Kinesis stream (the other option will be setting `KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE` to `TRIM_HORIZON`, which lets the consumer start reading the Kinesis stream from the earliest record possible). +Note that the consumer attempts to discover new Kinesis shards due to resharding at a fixed default interval of +10 seconds. In other words, new shards may take up to 10 seconds to be discovered. This setting can be overridden +to other values by setting a different value for `KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS` in the +supplied consumer configuration. + Other optional configuration keys can be found in `KinesisConfigConstants`. ### Kinesis Producer diff --git a/docs/apis/streaming/fault_tolerance.md b/docs/apis/streaming/fault_tolerance.md index 3edc65d0bd570..0e9863286dd4a 100644 --- a/docs/apis/streaming/fault_tolerance.md +++ b/docs/apis/streaming/fault_tolerance.md @@ -123,7 +123,7 @@ not for other sources. The following table lists the state update guarantees of AWS Kinesis Streams exactly once - Current version does not handle stream resharding + RabbitMQ diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java index 19b36e41af94c..25561c3b88bd1 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java @@ -22,14 +22,10 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kinesis.config.InitialPosition; -import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; -import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper; import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; @@ -37,31 +33,26 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; +import java.util.List; import java.util.Properties; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * The Flink Kinesis Consumer is a parallel streaming data source that pulls data from multiple AWS Kinesis streams - * within the same AWS service region. Each instance of the consumer is responsible for fetching data records from - * one or more Kinesis shards. + * The Flink Kinesis Consumer is an exactly-once parallel streaming data source that subscribes to multiple AWS Kinesis + * streams within the same AWS service region, and can handle resharding of streams. Each subtask of the consumer is + * responsible for fetching data records from multiple Kinesis shards. The number of shards fetched by each subtask will + * change as shards are closed and created by Kinesis. * *

To leverage Flink's checkpointing mechanics for exactly-once streaming processing guarantees, the Flink Kinesis * consumer is implemented with the AWS Java SDK, instead of the officially recommended AWS Kinesis Client Library, for * low-level control on the management of stream state. The Flink Kinesis Connector also supports setting the initial * starting points of Kinesis streams, namely TRIM_HORIZON and LATEST.

* - *

NOTE: The current implementation does not correctly handle resharding of AWS Kinesis streams.

- *

NOTE: Since Kinesis and Kafka share many common abstractions, the implementation is heavily based on - * the Flink Kafka Consumer.

- * * @param the type of data emitted */ public class FlinkKinesisConsumer extends RichParallelSourceFunction @@ -75,8 +66,8 @@ public class FlinkKinesisConsumer extends RichParallelSourceFunction // Consumer properties // ------------------------------------------------------------------------ - /** The complete list of shards */ - private final List shards; + /** The names of the Kinesis streams that we will be consuming from */ + private final List streams; /** Properties to parametrize settings such as AWS service region, initial position in stream, * shard list retrieval behaviours, etc */ @@ -90,16 +81,14 @@ public class FlinkKinesisConsumer extends RichParallelSourceFunction // ------------------------------------------------------------------------ /** Per-task fetcher for Kinesis data records, where each fetcher pulls data from one or more Kinesis shards */ - private transient KinesisDataFetcher fetcher; + private transient KinesisDataFetcher fetcher; - /** The sequence numbers of the last fetched data records from Kinesis by this task */ - private transient HashMap lastSequenceNums; + /** The sequence numbers in the last state snapshot of this subtask */ + private transient HashMap lastStateSnapshot; /** The sequence numbers to restore to upon restore from failure */ private transient HashMap sequenceNumsToRestore; - private volatile boolean hasAssignedShards; - private volatile boolean running = true; @@ -121,7 +110,7 @@ public class FlinkKinesisConsumer extends RichParallelSourceFunction * The properties used to configure AWS credentials, AWS region, and initial starting position. */ public FlinkKinesisConsumer(String stream, DeserializationSchema deserializer, Properties configProps) { - this(stream, new KinesisDeserializationSchemaWrapper(deserializer), configProps); + this(stream, new KinesisDeserializationSchemaWrapper<>(deserializer), configProps); } /** @@ -156,6 +145,9 @@ public FlinkKinesisConsumer(String stream, KinesisDeserializationSchema deser */ public FlinkKinesisConsumer(List streams, KinesisDeserializationSchema deserializer, Properties configProps) { checkNotNull(streams, "streams can not be null"); + checkArgument(streams.size() != 0, "must be consuming at least 1 stream"); + checkArgument(!streams.contains(""), "stream names cannot be empty Strings"); + this.streams = streams; this.configProps = checkNotNull(configProps, "configProps can not be null"); @@ -164,27 +156,12 @@ public FlinkKinesisConsumer(List streams, KinesisDeserializationSchema shardCountPerStream = new HashMap<>(); - for (KinesisStreamShard shard : shards) { - Integer shardCount = shardCountPerStream.get(shard.getStreamName()); - if (shardCount == null) { - shardCount = 1; - } else { - shardCount++; - } - shardCountPerStream.put(shard.getStreamName(), shardCount); - } StringBuilder sb = new StringBuilder(); - for (Map.Entry streamAndShardCountPair : shardCountPerStream.entrySet()) { - sb.append(streamAndShardCountPair.getKey()).append(" (").append(streamAndShardCountPair.getValue()).append("), "); + for (String stream : streams) { + sb.append(stream).append(", "); } - LOG.info("Flink Kinesis Consumer is going to read the following streams (with number of shards): {}", sb.toString()); + LOG.info("Flink Kinesis Consumer is going to read the following streams: {}", sb.toString()); } } @@ -196,111 +173,62 @@ public FlinkKinesisConsumer(List streams, KinesisDeserializationSchema assignedShards = assignShards(this.shards, numFlinkConsumerTasks, thisConsumerTaskIndex); - - // if there are no shards assigned to this consumer task, return without doing anything. - if (assignedShards.isEmpty()) { - LOG.info("Consumer task {} has no shards assigned to it", thisConsumerTaskIndex); - hasAssignedShards = false; - return; - } else { - hasAssignedShards = true; - } - - if (LOG.isInfoEnabled()) { - StringBuilder sb = new StringBuilder(); - for (KinesisStreamShard shard : assignedShards) { - sb.append(shard.getStreamName()).append(":").append(shard.getShardId()).append(", "); - } - LOG.info("Consumer task {} will read shards {} out of a total of {} shards", - thisConsumerTaskIndex, sb.toString(), this.shards.size()); - } - - fetcher = new KinesisDataFetcher(assignedShards, configProps, getRuntimeContext().getTaskName()); - // restore to the last known sequence numbers from the latest complete snapshot if (sequenceNumsToRestore != null) { if (LOG.isInfoEnabled()) { - LOG.info("Consumer task {} is restoring sequence numbers from previous checkpointed state", thisConsumerTaskIndex); - } - - for (Map.Entry restoreSequenceNum : sequenceNumsToRestore.entrySet()) { - // advance the corresponding shard to the last known sequence number - fetcher.advanceSequenceNumberTo(restoreSequenceNum.getKey(), restoreSequenceNum.getValue()); - } - - if (LOG.isInfoEnabled()) { - StringBuilder sb = new StringBuilder(); - for (Map.Entry restoreSequenceNo : sequenceNumsToRestore.entrySet()) { - KinesisStreamShard shard = restoreSequenceNo.getKey(); - sb.append(shard.getStreamName()).append(":").append(shard.getShardId()) - .append(" -> ").append(restoreSequenceNo.getValue()).append(", "); - } - LOG.info("Advanced the starting sequence numbers of consumer task {}: {}", thisConsumerTaskIndex, sb.toString()); + LOG.info("Subtask {} is restoring sequence numbers {} from previous checkpointed state", + getRuntimeContext().getIndexOfThisSubtask(), sequenceNumsToRestore.toString()); } // initialize sequence numbers with restored state - lastSequenceNums = sequenceNumsToRestore; + lastStateSnapshot = sequenceNumsToRestore; sequenceNumsToRestore = null; } else { // start fresh with empty sequence numbers if there are no snapshots to restore from. - lastSequenceNums = new HashMap<>(); - - // advance all assigned shards of this consumer task to either the earliest or latest sequence number, - // depending on the properties configuration (default is to set to latest sequence number). - InitialPosition initialPosition = InitialPosition.valueOf(configProps.getProperty( - KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, InitialPosition.LATEST.toString())); - - SentinelSequenceNumber sentinelSequenceNum; - switch (initialPosition) { - case TRIM_HORIZON: - sentinelSequenceNum = SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM; - break; - case LATEST: - default: - sentinelSequenceNum = SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM; - } - - for (KinesisStreamShard assignedShard : assignedShards) { - fetcher.advanceSequenceNumberTo(assignedShard, sentinelSequenceNum.get()); - } - - if (LOG.isInfoEnabled()) { - StringBuilder sb = new StringBuilder(); - for (KinesisStreamShard assignedShard : assignedShards) { - sb.append(assignedShard.getStreamName()).append(":").append(assignedShard.getShardId()) - .append(" -> ").append(sentinelSequenceNum.get()).append(", "); - } - LOG.info("Advanced the starting sequence numbers of consumer task {}: {}", thisConsumerTaskIndex, sb.toString()); - } + lastStateSnapshot = new HashMap<>(); } } @Override public void run(SourceContext sourceContext) throws Exception { - if (hasAssignedShards) { - fetcher.run(sourceContext, deserializer, lastSequenceNums); - } else { - // this source never completes because there is no assigned shards, - // so emit a Long.MAX_VALUE watermark to no block watermark forwarding - sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); - - final Object waitLock = new Object(); - while (running) { - try { - synchronized (waitLock) { - waitLock.wait(); - } - } catch (InterruptedException e) { - // do nothing + + // all subtasks will run a fetcher, regardless of whether or not the subtask will initially have + // shards to subscribe to; fetchers will continuously poll for changes in the shard list, so all subtasks + // can potentially have new shards to subscribe to later on + fetcher = new KinesisDataFetcher<>( + streams, sourceContext, getRuntimeContext(), configProps, deserializer); + + boolean isRestoringFromFailure = !lastStateSnapshot.isEmpty(); + fetcher.setIsRestoringFromFailure(isRestoringFromFailure); + + // if we are restoring from a checkpoint, we iterate over the restored + // state and accordingly seed the fetcher with subscribed shards states + if (isRestoringFromFailure) { + for (Map.Entry restored : lastStateSnapshot.entrySet()) { + fetcher.advanceLastDiscoveredShardOfStream( + restored.getKey().getStreamName(), restored.getKey().getShard().getShardId()); + + if (LOG.isInfoEnabled()) { + LOG.info("Subtask {} is seeding the fetcher with restored shard {}," + + "starting state set to the restored sequence number {}" + + getRuntimeContext().getIndexOfThisSubtask(), restored.getKey().toString(), restored.getValue()); } + fetcher.registerNewSubscribedShardState( + new KinesisStreamShardState(restored.getKey(), restored.getValue())); } } + // check that we are running before starting the fetcher + if (!running) { + return; + } + + // start the fetcher loop. The fetcher will stop running only when cancel() or + // close() is called, or an error is thrown by threads created by the fetcher + fetcher.runFetcher(); + + // check that the fetcher has terminated before fully closing + fetcher.awaitTermination(); sourceContext.close(); } @@ -308,13 +236,17 @@ public void run(SourceContext sourceContext) throws Exception { public void cancel() { running = false; - // interrupt the fetcher of any work KinesisDataFetcher fetcher = this.fetcher; this.fetcher = null; + + // this method might be called before the subtask actually starts running, + // so we must check if the fetcher is actually created if (fetcher != null) { try { - fetcher.close(); - } catch (IOException e) { + // interrupt the fetcher of any work + fetcher.shutdownFetcher(); + fetcher.awaitTermination(); + } catch (Exception e) { LOG.warn("Error while closing Kinesis data fetcher", e); } } @@ -337,11 +269,16 @@ public TypeInformation getProducedType() { @Override public HashMap snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - if (lastSequenceNums == null) { + if (lastStateSnapshot == null) { LOG.debug("snapshotState() requested on not yet opened source; returning null."); return null; } + if (fetcher == null) { + LOG.debug("snapshotState() requested on not yet running source; returning null."); + return null; + } + if (!running) { LOG.debug("snapshotState() called on closed source; returning null."); return null; @@ -351,56 +288,18 @@ public HashMap snapshotState(long checkpoint LOG.debug("Snapshotting state. ..."); } - @SuppressWarnings("unchecked") - HashMap currentSequenceNums = - (HashMap) lastSequenceNums.clone(); + lastStateSnapshot = fetcher.snapshotState(); - return currentSequenceNums; + if (LOG.isDebugEnabled()) { + LOG.debug("Snapshotting state. Last processed sequence numbers: {}, checkpoint id: {}, timestamp: {}", + lastStateSnapshot.toString(), checkpointId, checkpointTimestamp); + } + + return lastStateSnapshot; } @Override public void restoreState(HashMap restoredState) throws Exception { sequenceNumsToRestore = restoredState; } - - // ------------------------------------------------------------------------ - // Miscellaneous utilities - // ------------------------------------------------------------------------ - - /** - * Utility function to assign shards to a specific consumer task in a round-robin fashion. - */ - protected static List assignShards(List shards, int numFlinkConsumerTasks, int thisConsumerTaskIndex) { - checkArgument(numFlinkConsumerTasks > 0); - checkArgument(thisConsumerTaskIndex < numFlinkConsumerTasks); - - List closedShards = new ArrayList<>(); - List openShards = new ArrayList<>(); - - for (KinesisStreamShard shard : shards) { - if (shard.isClosed()) { - closedShards.add(shard); - } else { - openShards.add(shard); - } - } - - List subscribedShards = new ArrayList<>(); - - // separately round-robin assign open and closed shards so that all tasks have a fair chance of being - // assigned open shards (set of data records in closed shards are bounded) - - for (int i = 0; i < closedShards.size(); i++) { - if (i % numFlinkConsumerTasks == thisConsumerTaskIndex) { - subscribedShards.add(closedShards.get(i)); - } - } - - for (int i = 0; i < openShards.size(); i++) { - if (i % numFlinkConsumerTasks == thisConsumerTaskIndex) { - subscribedShards.add(openShards.get(i)); - } - } - return subscribedShards; - } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java index 29059ff58323f..1f7de7a459c38 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java @@ -26,15 +26,44 @@ public class KinesisConfigConstants { // Configuration Keys // ------------------------------------------------------------------------ - /** The max retries to retrieve metadata from a Kinesis stream using describeStream API - * (Note: describeStream attempts may be temporarily blocked due to AWS capping 5 attempts per sec) */ - public static final String CONFIG_STREAM_DESCRIBE_RETRIES = "flink.stream.describe.retry"; + /** The base backoff time between each describeStream attempt */ + public static final String CONFIG_STREAM_DESCRIBE_BACKOFF_BASE = "flink.stream.describe.backoff.base"; - /** The backoff time between each describeStream attempt */ - public static final String CONFIG_STREAM_DESCRIBE_BACKOFF = "flink.stream.describe.backoff"; + /** The maximum backoff time between each describeStream attempt */ + public static final String CONFIG_STREAM_DESCRIBE_BACKOFF_MAX = "flink.stream.describe.backoff.max"; + + /** The power constant for exponential backoff between each describeStream attempt */ + public static final String CONFIG_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = "flink.stream.describe.backoff.expconst"; /** The maximum number of records to try to get each time we fetch records from a AWS Kinesis shard */ - public static final String CONFIG_SHARD_RECORDS_PER_GET = "flink.shard.getrecords.maxcount"; + public static final String CONFIG_SHARD_GETRECORDS_MAX = "flink.shard.getrecords.maxrecordcount"; + + /** The maximum number of getRecords attempts if we get ProvisionedThroughputExceededException */ + public static final String CONFIG_SHARD_GETRECORDS_RETRIES = "flink.shard.getrecords.maxretries"; + + /** The base backoff time between getRecords attempts if we get a ProvisionedThroughputExceededException */ + public static final String CONFIG_SHARD_GETRECORDS_BACKOFF_BASE = "flink.shard.getrecords.backoff.base"; + + /** The maximum backoff time between getRecords attempts if we get a ProvisionedThroughputExceededException */ + public static final String CONFIG_SHARD_GETRECORDS_BACKOFF_MAX = "flink.shard.getrecords.backoff.max"; + + /** The power constant for exponential backoff between each getRecords attempt */ + public static final String CONFIG_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = "flink.shard.getrecords.backoff.expconst"; + + /** The maximum number of getShardIterator attempts if we get ProvisionedThroughputExceededException */ + public static final String CONFIG_SHARD_GETITERATOR_RETRIES = "flink.shard.getiterator.maxretries"; + + /** The base backoff time between getShardIterator attempts if we get a ProvisionedThroughputExceededException */ + public static final String CONFIG_SHARD_GETITERATOR_BACKOFF_BASE = "flink.shard.getiterator.backoff.base"; + + /** The maximum backoff time between getShardIterator attempts if we get a ProvisionedThroughputExceededException */ + public static final String CONFIG_SHARD_GETITERATOR_BACKOFF_MAX = "flink.shard.getiterator.backoff.max"; + + /** The power constant for exponential backoff between each getShardIterator attempt */ + public static final String CONFIG_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = "flink.shard.getiterator.backoff.expconst"; + + /** The interval between each attempt to discover new shards */ + public static final String CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS = "flink.shard.discovery.intervalmillis"; /** The initial position to start reading Kinesis streams from (LATEST is used if not set) */ public static final String CONFIG_STREAM_INIT_POSITION_TYPE = "flink.stream.initpos.type"; @@ -68,10 +97,30 @@ public class KinesisConfigConstants { // Default configuration values // ------------------------------------------------------------------------ - public static final int DEFAULT_STREAM_DESCRIBE_RETRY_TIMES = 3; + public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; + + public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L; + + public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; + + public static final int DEFAULT_SHARD_GETRECORDS_MAX = 100; + + public static final int DEFAULT_SHARD_GETRECORDS_RETRIES = 3; + + public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE = 300L; + + public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX = 1000L; + + public static final double DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; + + public static final int DEFAULT_SHARD_GETITERATOR_RETRIES = 3; + + public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE = 300L; + + public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX = 1000L; - public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF = 1000L; + public static final double DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; - public static final int DEFAULT_SHARD_RECORDS_PER_GET = 100; + public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L; } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 58933f63eeedb..269df556ba0fa 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -17,157 +17,553 @@ package org.apache.flink.streaming.connectors.kinesis.internals; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Map; -import java.util.HashMap; + +import java.util.LinkedList; import java.util.List; -import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards. - * The fetcher spawns a single thread for connection to each shard. + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following: + *
    + *
  • 1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset + * of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be + * subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe + * to the same subset of shards even after restoring)
  • + *
  • 2. decide where in each discovered shard should the fetcher start subscribing to
  • + *
  • 3. subscribe to shards by creating a single thread for each shard
  • + *
+ * + *

The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery), + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed + * by multiple threads, these operations should only be done using the handler methods provided in this class. */ -public class KinesisDataFetcher { +public class KinesisDataFetcher { private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class); - /** Config properties for the Flink Kinesis Consumer */ + // ------------------------------------------------------------------------ + // Consumer-wide settings + // ------------------------------------------------------------------------ + + /** Configuration properties for the Flink Kinesis Consumer */ private final Properties configProps; - /** The name of the consumer task that this fetcher was instantiated */ - private final String taskName; + /** The list of Kinesis streams that the consumer is subscribing to */ + private final List streams; + + /** + * The deserialization schema we will be using to convert Kinesis records to Flink objects. + * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must + * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}. + */ + private final KinesisDeserializationSchema deserializationSchema; + + // ------------------------------------------------------------------------ + // Subtask-specific settings + // ------------------------------------------------------------------------ + + /** Runtime context of the subtask that this fetcher was created in */ + private final RuntimeContext runtimeContext; + + private final int totalNumberOfConsumerSubtasks; + + private final int indexOfThisConsumerSubtask; + + /** + * This flag should be set by {@link FlinkKinesisConsumer} using + * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)} + */ + private boolean isRestoredFromFailure; + + // ------------------------------------------------------------------------ + // Executor services to run created threads + // ------------------------------------------------------------------------ - /** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */ - private HashMap assignedShardsWithStartingSequenceNum; + /** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */ + private final ExecutorService shardConsumersExecutor; - /** Reference to the thread that executed run() */ - private volatile Thread mainThread; + // ------------------------------------------------------------------------ + // Managed state, accessed and updated across multiple threads + // ------------------------------------------------------------------------ - /** Reference to the first error thrown by any of the spawned shard connection threads */ + /** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in. + * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called. + */ + private final Map subscribedStreamsToLastDiscoveredShardIds; + + /** + * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher + * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update + * the last processed sequence number of subscribed shards as they fetch and process records. + * + *

Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations + * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, + * all threads must use the following thread-safe methods this class provides to operate on this list: + *

    + *
  • {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}
  • + *
  • {@link KinesisDataFetcher#updateState(int, SequenceNumber)}
  • + *
  • {@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}
  • + *
+ */ + private final List subscribedShardsState; + + private final SourceFunction.SourceContext sourceContext; + + /** Checkpoint lock, also used to synchronize operations on subscribedShardsState */ + private final Object checkpointLock; + + /** Reference to the first error thrown by any of the {@link ShardConsumer} threads */ private final AtomicReference error; + /** The Kinesis proxy that the fetcher will be using to discover new shards */ + private final KinesisProxyInterface kinesis; + + /** Thread that executed runFetcher() */ + private Thread mainThread; + private volatile boolean running = true; /** - * Creates a new Kinesis Data Fetcher for the specified set of shards + * Creates a Kinesis Data Fetcher. * - * @param assignedShards the shards that this fetcher will pull data from - * @param configProps the configuration properties of this Flink Kinesis Consumer - * @param taskName the task name of this consumer task + * @param streams the streams to subscribe to + * @param sourceContext context of the source function + * @param runtimeContext this subtask's runtime context + * @param configProps the consumer configuration properties + * @param deserializationSchema deserialization schema */ - public KinesisDataFetcher(List assignedShards, Properties configProps, String taskName) { + public KinesisDataFetcher(List streams, + SourceFunction.SourceContext sourceContext, + RuntimeContext runtimeContext, + Properties configProps, + KinesisDeserializationSchema deserializationSchema) { + this(streams, + sourceContext, + sourceContext.getCheckpointLock(), + runtimeContext, + configProps, + deserializationSchema, + new AtomicReference(), + new LinkedList(), + createInitialSubscribedStreamsToLastDiscoveredShardsState(streams), + KinesisProxy.create(configProps)); + } + + /** This constructor is exposed for testing purposes */ + protected KinesisDataFetcher(List streams, + SourceFunction.SourceContext sourceContext, + Object checkpointLock, + RuntimeContext runtimeContext, + Properties configProps, + KinesisDeserializationSchema deserializationSchema, + AtomicReference error, + LinkedList subscribedShardsState, + HashMap subscribedStreamsToLastDiscoveredShardIds, + KinesisProxyInterface kinesis) { + this.streams = checkNotNull(streams); this.configProps = checkNotNull(configProps); - this.assignedShardsWithStartingSequenceNum = new HashMap<>(); - for (KinesisStreamShard shard : assignedShards) { - assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get()); - } - this.taskName = taskName; - this.error = new AtomicReference<>(); + this.sourceContext = checkNotNull(sourceContext); + this.checkpointLock = checkNotNull(checkpointLock); + this.runtimeContext = checkNotNull(runtimeContext); + this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks(); + this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask(); + this.deserializationSchema = checkNotNull(deserializationSchema); + this.kinesis = checkNotNull(kinesis); + + this.error = checkNotNull(error); + this.subscribedShardsState = checkNotNull(subscribedShardsState); + this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds); + + this.shardConsumersExecutor = + createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks()); } /** - * Advance a shard's starting sequence number to a specified value + * Starts the fetcher. After starting the fetcher, it can only + * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}. * - * @param streamShard the shard to perform the advance on - * @param sequenceNum the sequence number to advance to + * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher. */ - public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) { - if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) { - throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read."); - } - assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum); - } - - public void run(SourceFunction.SourceContext sourceContext, - KinesisDeserializationSchema deserializationSchema, - HashMap lastSequenceNums) throws Exception { + public void runFetcher() throws Exception { - if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) { - throw new IllegalArgumentException("No shards set to read for this fetcher"); + // check that we are running before proceeding + if (!running) { + return; } this.mainThread = Thread.currentThread(); - LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum); + // ------------------------------------------------------------------------ + // Procedures before starting the infinite while loop: + // ------------------------------------------------------------------------ + + // 1. query for any new shards that may have been created while the Kinesis consumer was not running, + // and register them to the subscribedShardState list. + List newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe(); + for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) { + // the starting state for new shards created while the consumer wasn't running depends on whether or not + // we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means + // all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint, + // any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards. + SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure) + ? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM + : KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps); - // create a thread for each individual shard - ArrayList> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size()); - for (Map.Entry assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) { - ShardConsumerThread thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(), - assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums); - thread.setName(String.format("ShardConsumer - %s - %s/%s", - taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId())); - thread.setDaemon(true); - consumerThreads.add(thread); + if (LOG.isInfoEnabled()) { + String logFormat = (isRestoredFromFailure) + ? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}" + : "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" + + "running due to failure, starting state set as sequence number {}"; + + LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get()); + } + registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get())); } - // check that we are viable for running for the last time before starting threads - if (!running) { - return; + // 2. check that there is at least one shard in the subscribed streams to consume from (can be done by + // checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null) + boolean hasShards = false; + StringBuilder streamsWithNoShardsFound = new StringBuilder(); + for (Map.Entry streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) { + if (streamToLastDiscoveredShardEntry.getValue() != null) { + hasShards = true; + } else { + streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", "); + } } - for (ShardConsumerThread shardConsumer : consumerThreads) { - LOG.info("Starting thread {}", shardConsumer.getName()); - shardConsumer.start(); + if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) { + LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}", + indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString()); } - // wait until all consumer threads are done, or until the fetcher is aborted, or until - // an error occurred in one of the consumer threads - try { - boolean consumersStillRunning = true; - while (running && error.get() == null && consumersStillRunning) { - try { - // wait for the consumer threads. if an error occurs, we are interrupted - for (ShardConsumerThread consumerThread : consumerThreads) { - consumerThread.join(); - } + if (!hasShards) { + throw new RuntimeException("No shards can be found for all subscribed streams: " + streams); + } - // check if there are consumer threads still running - consumersStillRunning = false; - for (ShardConsumerThread consumerThread : consumerThreads) { - consumersStillRunning = consumersStillRunning | consumerThread.isAlive(); - } - } catch (InterruptedException e) { - // ignore - } + // 3. start consuming any shard state we already have in the subscribedShardState up to this point; the + // subscribedShardState may already be seeded with values due to step 1., or explicitly added by the + // consumer using a restored state checkpoint + for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) { + KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex); + + if (LOG.isInfoEnabled()) { + LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" + + runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(), + seededShardState.getLastProcessedSequenceNum(), seededStateIndex); } - // make sure any asynchronous error is noticed - Throwable error = this.error.get(); - if (error != null) { - throw new Exception(error.getMessage(), error); + shardConsumersExecutor.submit( + new ShardConsumer<>( + this, + seededStateIndex, + subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(), + subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum())); + } + + // ------------------------------------------------------------------------ + + // finally, start the infinite shard discovery and consumer launching loop; + // we will escape from this loop only when shutdownFetcher() or stopWithError() is called + + final long discoveryIntervalMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS))); + + while (running) { + List newShardsDueToResharding = discoverNewShardsToSubscribe(); + + for (KinesisStreamShard shard : newShardsDueToResharding) { + // since there may be delay in discovering a new shard, all new shards due to + // resharding should be read starting from the earliest record possible + KinesisStreamShardState newShardState = + new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get()); + int newStateIndex = registerNewSubscribedShardState(newShardState); + + if (LOG.isInfoEnabled()) { + LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " + + "the shard from sequence number {} with ShardConsumer {}", + runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(), + newShardState.getLastProcessedSequenceNum(), newStateIndex); + } + + shardConsumersExecutor.submit( + new ShardConsumer<>( + this, + newStateIndex, + newShardState.getKinesisStreamShard(), + newShardState.getLastProcessedSequenceNum())); } - } finally { - for (ShardConsumerThread consumerThread : consumerThreads) { - if (consumerThread.isAlive()) { - consumerThread.cancel(); + + // we also check if we are running here so that we won't start the discovery sleep + // interval if the running flag was set to false during the middle of the while loop + if (running && discoveryIntervalMillis != 0) { + try { + Thread.sleep(discoveryIntervalMillis); + } catch (InterruptedException iex) { + // the sleep may be interrupted by shutdownFetcher() } } } + + // make sure all resources have been terminated before leaving + awaitTermination(); + + // any error thrown in the shard consumer threads will be thrown to the main thread + Throwable throwable = this.error.get(); + if (throwable != null) { + if (throwable instanceof Exception) { + throw (Exception) throwable; + } else if (throwable instanceof Error) { + throw (Error) throwable; + } else { + throw new Exception(throwable); + } + } + } + + /** + * Creates a snapshot of the current last processed sequence numbers of each subscribed shard. + * + * @return state snapshot + */ + public HashMap snapshotState() { + // this method assumes that the checkpoint lock is held + assert Thread.holdsLock(checkpointLock); + + HashMap stateSnapshot = new HashMap<>(); + for (KinesisStreamShardState shardWithState : subscribedShardsState) { + stateSnapshot.put(shardWithState.getKinesisStreamShard(), shardWithState.getLastProcessedSequenceNum()); + } + return stateSnapshot; } - public void close() throws IOException { - this.running = false; + /** + * Starts shutting down the fetcher. Must be called to allow {@link KinesisDataFetcher#runFetcher()} to complete. + * Once called, the shutdown procedure will be executed and all shard consuming threads will be interrupted. + */ + public void shutdownFetcher() { + running = false; + mainThread.interrupt(); // the main thread may be sleeping for the discovery interval + + if (LOG.isInfoEnabled()) { + LOG.info("Shutting down the shard consumer threads of subtask {} ...", + runtimeContext.getIndexOfThisSubtask()); + } + shardConsumersExecutor.shutdownNow(); + } + + /** After calling {@link KinesisDataFetcher#shutdownFetcher()}, this can be called to await the fetcher shutdown */ + public void awaitTermination() throws InterruptedException { + while(!shardConsumersExecutor.isTerminated()) { + Thread.sleep(50); + } } - public void stopWithError(Throwable throwable) { + /** Called by created threads to pass on errors. Only the first thrown error is set. + * Once set, the shutdown process will be executed and all shard consuming threads will be interrupted. */ + protected void stopWithError(Throwable throwable) { if (this.error.compareAndSet(null, throwable)) { - if (mainThread != null) { - mainThread.interrupt(); + shutdownFetcher(); + } + } + + // ------------------------------------------------------------------------ + // Functions that update the subscribedStreamToLastDiscoveredShardIds state + // ------------------------------------------------------------------------ + + /** Updates the last discovered shard of a subscribed stream; only updates if the update is valid */ + public void advanceLastDiscoveredShardOfStream(String stream, String shardId) { + String lastSeenShardIdOfStream = this.subscribedStreamsToLastDiscoveredShardIds.get(stream); + + // the update is valid only if the given shard id is greater + // than the previous last seen shard id of the stream + if (lastSeenShardIdOfStream == null) { + // if not previously set, simply put as the last seen shard id + this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); + } else if (KinesisStreamShard.compareShardIds(shardId, lastSeenShardIdOfStream) > 0) { + this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); + } + } + + /** + * A utility function that does the following: + * + * 1. Find new shards for each stream that we haven't seen before + * 2. For each new shard, determine whether this consumer subtask should subscribe to them; + * if yes, it is added to the returned list of shards + * 3. Update the subscribedStreamsToLastDiscoveredShardIds state so that we won't get shards + * that we have already seen before the next time this function is called + */ + private List discoverNewShardsToSubscribe() throws InterruptedException { + + List newShardsToSubscribe = new LinkedList<>(); + + GetShardListResult shardListResult = kinesis.getShardList(subscribedStreamsToLastDiscoveredShardIds); + if (shardListResult.hasRetrievedShards()) { + Set streamsWithNewShards = shardListResult.getStreamsWithRetrievedShards(); + + for (String stream : streamsWithNewShards) { + List newShardsOfStream = shardListResult.getRetrievedShardListOfStream(stream); + for (KinesisStreamShard newShard : newShardsOfStream) { + if (isThisSubtaskShouldSubscribeTo(newShard, totalNumberOfConsumerSubtasks, indexOfThisConsumerSubtask)) { + newShardsToSubscribe.add(newShard); + } + } + + advanceLastDiscoveredShardOfStream( + stream, shardListResult.getLastSeenShardOfStream(stream).getShard().getShardId()); } } + + return newShardsToSubscribe; + } + + // ------------------------------------------------------------------------ + // Functions to get / set information about the consumer + // ------------------------------------------------------------------------ + + public void setIsRestoringFromFailure(boolean bool) { + this.isRestoredFromFailure = bool; + } + + protected Properties getConsumerConfiguration() { + return configProps; + } + + protected KinesisDeserializationSchema getClonedDeserializationSchema() { + try { + return InstantiationUtil.clone(deserializationSchema, runtimeContext.getUserCodeClassLoader()); + } catch (IOException | ClassNotFoundException ex) { + // this really shouldn't happen; simply wrap it around a runtime exception + throw new RuntimeException(ex); + } + } + + // ------------------------------------------------------------------------ + // Thread-safe operations for record emitting and shard state updating + // that assure atomicity with respect to the checkpoint lock + // ------------------------------------------------------------------------ + + /** + * Atomic operation to collect a record and update state to the sequence number of the record. + * This method is called by {@link ShardConsumer}s. + * + * @param record the record to collect + * @param shardStateIndex index of the shard to update in subscribedShardsState; + * this index should be the returned value from + * {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called + * when the shard state was registered. + * @param lastSequenceNumber the last sequence number value to update + */ + protected void emitRecordAndUpdateState(T record, int shardStateIndex, SequenceNumber lastSequenceNumber) { + synchronized (checkpointLock) { + sourceContext.collect(record); + updateState(shardStateIndex, lastSequenceNumber); + } + } + + /** + * Update the shard to last processed sequence number state. + * This method is called by {@link ShardConsumer}s. + * + * @param shardStateIndex index of the shard to update in subscribedShardsState; + * this index should be the returned value from + * {@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}, called + * when the shard state was registered. + * @param lastSequenceNumber the last sequence number value to update + */ + protected void updateState(int shardStateIndex, SequenceNumber lastSequenceNumber) { + synchronized (checkpointLock) { + subscribedShardsState.get(shardStateIndex).setLastProcessedSequenceNum(lastSequenceNumber); + } + } + + /** + * Register a new subscribed shard state. + * + * @param newSubscribedShardState the new shard state that this fetcher is to be subscribed to + */ + public int registerNewSubscribedShardState(KinesisStreamShardState newSubscribedShardState) { + + synchronized (checkpointLock) { + subscribedShardsState.add(newSubscribedShardState); + return subscribedShardsState.size()-1; + } + } + + // ------------------------------------------------------------------------ + // Miscellaneous utility functions + // ------------------------------------------------------------------------ + + /** + * Utility function to determine whether a shard should be subscribed by this consumer subtask. + * + * @param shard the shard to determine + * @param totalNumberOfConsumerSubtasks total number of consumer subtasks + * @param indexOfThisConsumerSubtask index of this consumer subtask + */ + private static boolean isThisSubtaskShouldSubscribeTo(KinesisStreamShard shard, + int totalNumberOfConsumerSubtasks, + int indexOfThisConsumerSubtask) { + return (Math.abs(shard.hashCode() % totalNumberOfConsumerSubtasks)) == indexOfThisConsumerSubtask; + } + + private static ExecutorService createShardConsumersThreadPool(final String subtaskName) { + return Executors.newCachedThreadPool(new ThreadFactory() { + @Override + public Thread newThread(Runnable runnable) { + final AtomicLong threadCount = new AtomicLong(0); + Thread thread = new Thread(runnable); + thread.setName("shardConsumers-" + subtaskName + "-thread-" + threadCount.getAndIncrement()); + thread.setDaemon(true); + return thread; + } + }); + } + + /** + * Utility function to create an initial map of the last discovered shard id of each subscribed stream, set to null; + * This is called in the constructor; correct values will be set later on by calling advanceLastDiscoveredShardOfStream() + * + * @param streams the list of subscribed streams + * @return the initial map for subscribedStreamsToLastDiscoveredShardIds + */ + protected static HashMap createInitialSubscribedStreamsToLastDiscoveredShardsState(List streams) { + HashMap initial = new HashMap<>(); + for (String stream : streams) { + initial.put(stream, null); + } + return initial; } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java similarity index 54% rename from flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java rename to flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java index 66452e6b1cb81..262c8898c5cdb 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThread.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java @@ -21,18 +21,17 @@ import com.amazonaws.services.kinesis.model.GetRecordsResult; import com.amazonaws.services.kinesis.model.Record; import com.amazonaws.services.kinesis.model.ShardIteratorType; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; import java.io.IOException; import java.math.BigInteger; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.List; import java.util.Properties; @@ -41,40 +40,59 @@ /** * Thread that does the actual data pulling from AWS Kinesis shards. Each thread is in charge of one Kinesis shard only. */ -public class ShardConsumerThread extends Thread { - private final SourceFunction.SourceContext sourceContext; +public class ShardConsumer implements Runnable { + private final KinesisDeserializationSchema deserializer; - private final HashMap seqNoState; - private final KinesisProxy kinesisProxy; + private final KinesisProxyInterface kinesis; + + private final int subscribedShardStateIndex; - private final KinesisDataFetcher ownerRef; + private final KinesisDataFetcher fetcherRef; - private final KinesisStreamShard assignedShard; + private final KinesisStreamShard subscribedShard; private final int maxNumberOfRecordsPerFetch; private SequenceNumber lastSequenceNum; - private volatile boolean running = true; + /** + * Creates a shard consumer. + * + * @param fetcherRef reference to the owning fetcher + * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to + * @param subscribedShard the shard this consumer is subscribed to + * @param lastSequenceNum the sequence number in the shard to start consuming + */ + public ShardConsumer(KinesisDataFetcher fetcherRef, + Integer subscribedShardStateIndex, + KinesisStreamShard subscribedShard, + SequenceNumber lastSequenceNum) { + this(fetcherRef, + subscribedShardStateIndex, + subscribedShard, + lastSequenceNum, + KinesisProxy.create(fetcherRef.getConsumerConfiguration())); + } - public ShardConsumerThread(KinesisDataFetcher ownerRef, - Properties props, - KinesisStreamShard assignedShard, + /** This constructor is exposed for testing purposes */ + protected ShardConsumer(KinesisDataFetcher fetcherRef, + Integer subscribedShardStateIndex, + KinesisStreamShard subscribedShard, SequenceNumber lastSequenceNum, - SourceFunction.SourceContext sourceContext, - KinesisDeserializationSchema deserializer, - HashMap seqNumState) { - this.ownerRef = checkNotNull(ownerRef); - this.assignedShard = checkNotNull(assignedShard); + KinesisProxyInterface kinesis) { + this.fetcherRef = checkNotNull(fetcherRef); + this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex); + this.subscribedShard = checkNotNull(subscribedShard); this.lastSequenceNum = checkNotNull(lastSequenceNum); - this.sourceContext = checkNotNull(sourceContext); - this.deserializer = checkNotNull(deserializer); - this.seqNoState = checkNotNull(seqNumState); - this.kinesisProxy = new KinesisProxy(props); - this.maxNumberOfRecordsPerFetch = Integer.valueOf(props.getProperty( - KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET, - Integer.toString(KinesisConfigConstants.DEFAULT_SHARD_RECORDS_PER_GET))); + + this.deserializer = fetcherRef.getClonedDeserializationSchema(); + + Properties consumerConfig = fetcherRef.getConsumerConfiguration(); + this.kinesis = kinesis; + this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_MAX, + Integer.toString(KinesisConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX))); } @SuppressWarnings("unchecked") @@ -87,13 +105,13 @@ public void run() { if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) { // if the shard is already closed, there will be no latest next record to get for this shard - if (assignedShard.isClosed()) { + if (subscribedShard.isClosed()) { nextShardItr = null; } else { - nextShardItr = kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.LATEST.toString(), null); + nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.LATEST.toString(), null); } } else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) { - nextShardItr = kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.TRIM_HORIZON.toString(), null); + nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.TRIM_HORIZON.toString(), null); } else if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) { nextShardItr = null; } else { @@ -103,22 +121,22 @@ public void run() { if (lastSequenceNum.isAggregated()) { String itrForLastAggregatedRecord = - kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber()); + kinesis.getShardIterator(subscribedShard, ShardIteratorType.AT_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber()); // get only the last aggregated record - GetRecordsResult getRecordsResult = kinesisProxy.getRecords(itrForLastAggregatedRecord, 1); + GetRecordsResult getRecordsResult = kinesis.getRecords(itrForLastAggregatedRecord, 1); List fetchedRecords = deaggregateRecords( - getRecordsResult.getRecords(), - assignedShard.getStartingHashKey(), - assignedShard.getEndingHashKey()); + getRecordsResult.getRecords(), + subscribedShard.getShard().getHashKeyRange().getStartingHashKey(), + subscribedShard.getShard().getHashKeyRange().getEndingHashKey()); long lastSubSequenceNum = lastSequenceNum.getSubSequenceNumber(); for (UserRecord record : fetchedRecords) { // we have found a dangling sub-record if it has a larger subsequence number // than our last sequence number; if so, collect the record and update state if (record.getSubSequenceNumber() > lastSubSequenceNum) { - collectRecordAndUpdateState(record); + deserializeRecordForCollectionAndUpdateState(record); } } @@ -126,44 +144,49 @@ public void run() { nextShardItr = getRecordsResult.getNextShardIterator(); } else { // the last record was non-aggregated, so we can simply start from the next record - nextShardItr = kinesisProxy.getShardIterator(assignedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber()); + nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber()); } } - while(running) { + while(isRunning()) { if (nextShardItr == null) { - synchronized (sourceContext.getCheckpointLock()) { - seqNoState.put(assignedShard, SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()); - } + fetcherRef.updateState(subscribedShardStateIndex, SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()); + // we can close this consumer thread once we've reached the end of the subscribed shard break; } else { - GetRecordsResult getRecordsResult = kinesisProxy.getRecords(nextShardItr, maxNumberOfRecordsPerFetch); + GetRecordsResult getRecordsResult = kinesis.getRecords(nextShardItr, maxNumberOfRecordsPerFetch); // each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding List fetchedRecords = deaggregateRecords( getRecordsResult.getRecords(), - assignedShard.getStartingHashKey(), - assignedShard.getEndingHashKey()); + subscribedShard.getShard().getHashKeyRange().getStartingHashKey(), + subscribedShard.getShard().getHashKeyRange().getEndingHashKey()); for (UserRecord record : fetchedRecords) { - collectRecordAndUpdateState(record); + deserializeRecordForCollectionAndUpdateState(record); } nextShardItr = getRecordsResult.getNextShardIterator(); } } } catch (Throwable t) { - ownerRef.stopWithError(t); + fetcherRef.stopWithError(t); } } - public void cancel() { - this.running = false; - this.interrupt(); + /** + * The loop in run() checks this before fetching next batch of records. Since this runnable will be executed + * by the ExecutorService {@link KinesisDataFetcher#shardConsumersExecutor}, the only way to close down this thread + * would be by calling shutdownNow() on {@link KinesisDataFetcher#shardConsumersExecutor} and let the executor service + * interrupt all currently running {@link ShardConsumer}s. + */ + private boolean isRunning() { + return !Thread.interrupted(); } - private void collectRecordAndUpdateState(UserRecord record) throws IOException { + private void deserializeRecordForCollectionAndUpdateState(UserRecord record) + throws IOException { ByteBuffer recordData = record.getData(); byte[] dataBytes = new byte[recordData.remaining()]; @@ -171,18 +194,19 @@ private void collectRecordAndUpdateState(UserRecord record) throws IOException { byte[] keyBytes = record.getPartitionKey().getBytes(); - final T value = deserializer.deserialize(keyBytes, dataBytes, assignedShard.getStreamName(), + final T value = deserializer.deserialize(keyBytes, dataBytes, subscribedShard.getStreamName(), record.getSequenceNumber()); - synchronized (sourceContext.getCheckpointLock()) { - sourceContext.collect(value); - if (record.isAggregated()) { - seqNoState.put( - assignedShard, - new SequenceNumber(record.getSequenceNumber(), record.getSubSequenceNumber())); - } else { - seqNoState.put(assignedShard, new SequenceNumber(record.getSequenceNumber())); - } + if (record.isAggregated()) { + fetcherRef.emitRecordAndUpdateState( + value, + subscribedShardStateIndex, + new SequenceNumber(record.getSequenceNumber(), record.getSubSequenceNumber())); + } else { + fetcherRef.emitRecordAndUpdateState( + value, + subscribedShardStateIndex, + new SequenceNumber(record.getSequenceNumber())); } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java index b21c5bba491bc..53ed11b1b7d97 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java @@ -49,51 +49,20 @@ public KinesisStreamShard(String streamName, Shard shard) { this.streamName = checkNotNull(streamName); this.shard = checkNotNull(shard); - this.cachedHash = 37 * (streamName.hashCode() + shard.hashCode()); + // since our description of Kinesis Streams shards can be fully defined with the stream name and shard id, + // our hash doesn't need to use hash code of Amazon's description of Shards, which uses other info for calculation + int hash = 17; + hash = 37 * hash + streamName.hashCode(); + hash = 37 * hash + shard.getShardId().hashCode(); + this.cachedHash = hash; } public String getStreamName() { return streamName; } - public String getShardId() { - return shard.getShardId(); - } - - public String getStartingSequenceNumber() { - return shard.getSequenceNumberRange().getStartingSequenceNumber(); - } - - public String getEndingSequenceNumber() { - return shard.getSequenceNumberRange().getEndingSequenceNumber(); - } - - public String getStartingHashKey() { - return shard.getHashKeyRange().getStartingHashKey(); - } - - public String getEndingHashKey() { - return shard.getHashKeyRange().getEndingHashKey(); - } - public boolean isClosed() { - return (getEndingSequenceNumber() != null); - } - - public String getParentShardId() { - return shard.getParentShardId(); - } - - public String getAdjacentParentShardId() { - return shard.getAdjacentParentShardId(); - } - - public boolean isSplitShard() { - return (getParentShardId() != null && getAdjacentParentShardId() == null); - } - - public boolean isMergedShard() { - return (getParentShardId() != null && getAdjacentParentShardId() != null); + return (shard.getSequenceNumberRange().getEndingSequenceNumber() != null); } public Shard getShard() { @@ -104,13 +73,7 @@ public Shard getShard() { public String toString() { return "KinesisStreamShard{" + "streamName='" + streamName + "'" + - ", shardId='" + getShardId() + "'" + - ", parentShardId='" + getParentShardId() + "'" + - ", adjacentParentShardId='" + getAdjacentParentShardId() + "'" + - ", startingSequenceNumber='" + getStartingSequenceNumber() + "'" + - ", endingSequenceNumber='" + getEndingSequenceNumber() + "'" + - ", startingHashKey='" + getStartingHashKey() + "'" + - ", endingHashKey='" + getEndingHashKey() + "'}"; + ", shard='" + shard.toString() + "'}"; } @Override @@ -132,4 +95,39 @@ public boolean equals(Object obj) { public int hashCode() { return cachedHash; } + + /** + * Utility function to compare two shard ids + * + * @param firstShardId first shard id to compare + * @param secondShardId second shard id to compare + * @return a value less than 0 if the first shard id is smaller than the second shard id, + * or a value larger than 0 the first shard is larger then the second shard id, + * or 0 if they are equal + */ + public static int compareShardIds(String firstShardId, String secondShardId) { + if (!isValidShardId(firstShardId)) { + throw new IllegalArgumentException("The first shard id has invalid format."); + } + + if (!isValidShardId(secondShardId)) { + throw new IllegalArgumentException("The second shard id has invalid format."); + } + + // digit segment of the shard id starts at index 8 + return Long.compare(Long.parseLong(firstShardId.substring(8)), Long.parseLong(secondShardId.substring(8))); + } + + /** + * Checks if a shard id has valid format. + * Kinesis stream shard ids have 12-digit numbers left-padded with 0's, + * prefixed with "shardId-", ex. "shardId-000000000015". + * + * @param shardId the shard id to check + * @return whether the shard id is valid + */ + public static boolean isValidShardId(String shardId) { + if (shardId == null) { return false; } + return shardId.matches("^shardId-\\d{12}"); + } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java new file mode 100644 index 0000000000000..00181da149a56 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShardState.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.model; + +/** + * A wrapper class that bundles a {@link KinesisStreamShard} with its last processed sequence number. + */ +public class KinesisStreamShardState { + + private KinesisStreamShard kinesisStreamShard; + private SequenceNumber lastProcessedSequenceNum; + + public KinesisStreamShardState(KinesisStreamShard kinesisStreamShard, SequenceNumber lastProcessedSequenceNum) { + this.kinesisStreamShard = kinesisStreamShard; + this.lastProcessedSequenceNum = lastProcessedSequenceNum; + } + + public KinesisStreamShard getKinesisStreamShard() { + return this.kinesisStreamShard; + } + + public SequenceNumber getLastProcessedSequenceNum() { + return this.lastProcessedSequenceNum; + } + + public void setLastProcessedSequenceNum(SequenceNumber update) { + this.lastProcessedSequenceNum = update; + } + + @Override + public String toString() { + return "KinesisStreamShardState{" + + "kinesisStreamShard='" + kinesisStreamShard.toString() + "'" + + ", lastProcessedSequenceNumber='" + lastProcessedSequenceNum.toString() + "'}"; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof KinesisStreamShardState)) { + return false; + } + + if (obj == this) { + return true; + } + + KinesisStreamShardState other = (KinesisStreamShardState) obj; + + return kinesisStreamShard.equals(other.getKinesisStreamShard()) && lastProcessedSequenceNum.equals(other.getLastProcessedSequenceNum()); + } + + @Override + public int hashCode() { + return 37 * (kinesisStreamShard.hashCode() + lastProcessedSequenceNum.hashCode()); + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java index 55752f803bbaa..8182201897da5 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/SentinelSequenceNumber.java @@ -27,10 +27,6 @@ */ public enum SentinelSequenceNumber { - /** Flag value to indicate that the sequence number of a shard is not set. This value is used - * as an initial value in {@link KinesisDataFetcher}'s constructor for all shard's sequence number. */ - SENTINEL_SEQUENCE_NUMBER_NOT_SET( new SequenceNumber("SEQUENCE_NUMBER_NOT_SET") ), - /** Flag value for shard's sequence numbers to indicate that the * shard should start to be read from the latest incoming records */ SENTINEL_LATEST_SEQUENCE_NUM( new SequenceNumber("LATEST_SEQUENCE_NUM") ), diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java new file mode 100644 index 0000000000000..04b165441f3de --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/GetShardListResult.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.proxy; + +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; + +import java.util.LinkedList; +import java.util.List; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Basic model class to bundle the shards retrieved from Kinesis on a {@link KinesisProxyInterface#getShardList(Map)} call. + */ +public class GetShardListResult { + + private final Map> streamsToRetrievedShardList = new HashMap<>(); + + public void addRetrievedShardToStream(String stream, KinesisStreamShard retrievedShard) { + if (!streamsToRetrievedShardList.containsKey(stream)) { + streamsToRetrievedShardList.put(stream, new LinkedList()); + } + streamsToRetrievedShardList.get(stream).add(retrievedShard); + } + + public void addRetrievedShardsToStream(String stream, List retrievedShards) { + if (retrievedShards.size() != 0) { + if (!streamsToRetrievedShardList.containsKey(stream)) { + streamsToRetrievedShardList.put(stream, new LinkedList()); + } + streamsToRetrievedShardList.get(stream).addAll(retrievedShards); + } + } + + public List getRetrievedShardListOfStream(String stream) { + if (!streamsToRetrievedShardList.containsKey(stream)) { + return null; + } else { + return streamsToRetrievedShardList.get(stream); + } + } + + public KinesisStreamShard getLastSeenShardOfStream(String stream) { + if (!streamsToRetrievedShardList.containsKey(stream)) { + return null; + } else { + return streamsToRetrievedShardList.get(stream).getLast(); + } + } + + public boolean hasRetrievedShards() { + return !streamsToRetrievedShardList.isEmpty(); + } + + public Set getStreamsWithRetrievedShards() { + return streamsToRetrievedShardList.keySet(); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index d035c033dbf01..22f667e845164 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -17,21 +17,17 @@ package org.apache.flink.streaming.connectors.kinesis.proxy; -import com.amazonaws.ClientConfiguration; -import com.amazonaws.ClientConfigurationFactory; -import com.amazonaws.regions.Region; -import com.amazonaws.regions.Regions; import com.amazonaws.services.kinesis.AmazonKinesisClient; import com.amazonaws.services.kinesis.model.DescribeStreamRequest; import com.amazonaws.services.kinesis.model.DescribeStreamResult; import com.amazonaws.services.kinesis.model.GetRecordsRequest; import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.GetShardIteratorResult; import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; import com.amazonaws.services.kinesis.model.LimitExceededException; import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.StreamStatus; import com.amazonaws.services.kinesis.model.Shard; -import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; @@ -41,174 +37,288 @@ import java.util.ArrayList; import java.util.List; import java.util.Properties; +import java.util.Map; +import java.util.Random; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * A utility class that is used as a proxy to make calls to AWS Kinesis - * for several functions, such as getting a list of shards and fetching - * a batch of data records starting from a specified record sequence number. + * Kinesis proxy implementation - a utility class that is used as a proxy to make + * calls to AWS Kinesis for several functions, such as getting a list of shards and + * fetching a batch of data records starting from a specified record sequence number. * * NOTE: * In the AWS KCL library, there is a similar implementation - {@link com.amazonaws.services.kinesis.clientlibrary.proxies.KinesisProxy}. * This implementation differs mainly in that we can make operations to arbitrary Kinesis streams, which is a needed * functionality for the Flink Kinesis Connecter since the consumer may simultaneously read from multiple Kinesis streams. */ -public class KinesisProxy { +public class KinesisProxy implements KinesisProxyInterface { private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class); /** The actual Kinesis client from the AWS SDK that we will be using to make calls */ private final AmazonKinesisClient kinesisClient; - /** Configuration properties of this Flink Kinesis Connector */ - private final Properties configProps; + /** Random seed used to calculate backoff jitter for Kinesis operations */ + private final static Random seed = new Random(); + + // ------------------------------------------------------------------------ + // describeStream() related performance settings + // ------------------------------------------------------------------------ + + /** Base backoff millis for the describe stream operation */ + private final long describeStreamBaseBackoffMillis; + + /** Maximum backoff millis for the describe stream operation */ + private final long describeStreamMaxBackoffMillis; + + /** Exponential backoff power constant for the describe stream operation */ + private final double describeStreamExpConstant; + + // ------------------------------------------------------------------------ + // getRecords() related performance settings + // ------------------------------------------------------------------------ + + /** Base backoff millis for the get records operation */ + private final long getRecordsBaseBackoffMillis; + + /** Maximum backoff millis for the get records operation */ + private final long getRecordsMaxBackoffMillis; + + /** Exponential backoff power constant for the get records operation */ + private final double getRecordsExpConstant; + + /** Maximum attempts for the get records operation */ + private final int getRecordsMaxAttempts; + + // ------------------------------------------------------------------------ + // getShardIterator() related performance settings + // ------------------------------------------------------------------------ + + /** Base backoff millis for the get shard iterator operation */ + private final long getShardIteratorBaseBackoffMillis; + + /** Maximum backoff millis for the get shard iterator operation */ + private final long getShardIteratorMaxBackoffMillis; + + /** Exponential backoff power constant for the get shard iterator operation */ + private final double getShardIteratorExpConstant; + + /** Maximum attempts for the get shard iterator operation */ + private final int getShardIteratorMaxAttempts; /** * Create a new KinesisProxy based on the supplied configuration properties * * @param configProps configuration properties containing AWS credential and AWS region info */ - public KinesisProxy(Properties configProps) { - this.configProps = checkNotNull(configProps); + private KinesisProxy(Properties configProps) { + checkNotNull(configProps); + + this.kinesisClient = AWSUtil.createKinesisClient(configProps); + + this.describeStreamBaseBackoffMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_BASE, + Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE))); + this.describeStreamMaxBackoffMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_MAX, + Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX))); + this.describeStreamExpConstant = Double.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); - /* The AWS region that this proxy will be making calls to */ - String regionId = configProps.getProperty(KinesisConfigConstants.CONFIG_AWS_REGION); - // set Flink as a user agent - ClientConfiguration config = new ClientConfigurationFactory().getConfig(); - config.setUserAgent("Apache Flink " + EnvironmentInformation.getVersion() + " (" + EnvironmentInformation.getRevisionInformation().commitId + ") Kinesis Connector"); - AmazonKinesisClient client = new AmazonKinesisClient(AWSUtil.getCredentialsProvider(configProps).getCredentials(), config); + this.getRecordsBaseBackoffMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_BASE, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE))); + this.getRecordsMaxBackoffMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_MAX, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX))); + this.getRecordsExpConstant = Double.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(KinesisConfigConstants.DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT))); + this.getRecordsMaxAttempts = Integer.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_RETRIES, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_GETRECORDS_RETRIES))); - client.setRegion(Region.getRegion(Regions.fromName(regionId))); + this.getShardIteratorBaseBackoffMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_BASE, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE))); + this.getShardIteratorMaxBackoffMillis = Long.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_MAX, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX))); + this.getShardIteratorExpConstant = Double.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(KinesisConfigConstants.DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT))); + this.getShardIteratorMaxAttempts = Integer.valueOf( + configProps.getProperty( + KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_RETRIES, + Long.toString(KinesisConfigConstants.DEFAULT_SHARD_GETITERATOR_RETRIES))); - this.kinesisClient = client; } /** - * Get the next batch of data records using a specific shard iterator + * Creates a Kinesis proxy. * - * @param shardIterator a shard iterator that encodes info about which shard to read and where to start reading - * @param maxRecordsToGet the maximum amount of records to retrieve for this batch - * @return the batch of retrieved records + * @param configProps configuration properties + * @return the created kinesis proxy + */ + public static KinesisProxyInterface create(Properties configProps) { + return new KinesisProxy(configProps); + } + + /** + * {@inheritDoc} */ - public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) { + @Override + public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException { final GetRecordsRequest getRecordsRequest = new GetRecordsRequest(); getRecordsRequest.setShardIterator(shardIterator); getRecordsRequest.setLimit(maxRecordsToGet); GetRecordsResult getRecordsResult = null; - int remainingRetryTimes = Integer.valueOf( - configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES))); - long describeStreamBackoffTimeInMillis = Long.valueOf( - configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF))); - - int i=0; - while (i <= remainingRetryTimes && getRecordsResult == null) { + int attempt = 0; + while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) { try { getRecordsResult = kinesisClient.getRecords(getRecordsRequest); } catch (ProvisionedThroughputExceededException ex) { + long backoffMillis = fullJitterBackoff( + getRecordsBaseBackoffMillis, getRecordsMaxBackoffMillis, getRecordsExpConstant, attempt++); LOG.warn("Got ProvisionedThroughputExceededException. Backing off for " - + describeStreamBackoffTimeInMillis + " millis."); - try { - Thread.sleep(describeStreamBackoffTimeInMillis); - } catch (InterruptedException interruptEx) { - // - } + + backoffMillis + " millis."); + Thread.sleep(backoffMillis); } - i++; } if (getRecordsResult == null) { - throw new RuntimeException("Rate Exceeded"); + throw new RuntimeException("Rate Exceeded for getRecords operation - all " + getRecordsMaxAttempts + "retry" + + "attempts returned ProvisionedThroughputExceededException."); } return getRecordsResult; } /** - * Get the list of shards associated with multiple Kinesis streams - * - * @param streamNames the list of Kinesis streams - * @return a list of {@link KinesisStreamShard}s + * {@inheritDoc} */ - public List getShardList(List streamNames) { - List shardList = new ArrayList<>(); + @Override + public GetShardListResult getShardList(Map streamNamesWithLastSeenShardIds) throws InterruptedException { + GetShardListResult result = new GetShardListResult(); - for (String stream : streamNames) { - DescribeStreamResult describeStreamResult; - String lastSeenShardId = null; - - do { - describeStreamResult = describeStream(stream, lastSeenShardId); - - List shards = describeStreamResult.getStreamDescription().getShards(); - for (Shard shard : shards) { - shardList.add(new KinesisStreamShard(stream, shard)); - } - lastSeenShardId = shards.get(shards.size() - 1).getShardId(); - } while (describeStreamResult.getStreamDescription().isHasMoreShards()); + for (Map.Entry streamNameWithLastSeenShardId : streamNamesWithLastSeenShardIds.entrySet()) { + String stream = streamNameWithLastSeenShardId.getKey(); + String lastSeenShardId = streamNameWithLastSeenShardId.getValue(); + result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, lastSeenShardId)); } - return shardList; + return result; } /** - * Get a shard iterator for a Kinesis shard - * - * @param shard the shard to get the iterator for - * @param shardIteratorType the iterator type to get - * @param startingSeqNum the sequence number that the iterator will start from - * @return the shard iterator + * {@inheritDoc} */ - public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) { - return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator(); + @Override + public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) throws InterruptedException { + GetShardIteratorResult getShardIteratorResult = null; + + int attempt = 0; + while (attempt <= getShardIteratorMaxAttempts && getShardIteratorResult == null) { + try { + getShardIteratorResult = + kinesisClient.getShardIterator(shard.getStreamName(), shard.getShard().getShardId(), shardIteratorType, startingSeqNum); + } catch (ProvisionedThroughputExceededException ex) { + long backoffMillis = fullJitterBackoff( + getShardIteratorBaseBackoffMillis, getShardIteratorMaxBackoffMillis, getShardIteratorExpConstant, attempt++); + LOG.warn("Got ProvisionedThroughputExceededException. Backing off for " + + backoffMillis + " millis."); + Thread.sleep(backoffMillis); + } + } + + if (getShardIteratorResult == null) { + throw new RuntimeException("Rate Exceeded for getShardIterator operation - all " + getShardIteratorMaxAttempts + "retry" + + "attempts returned ProvisionedThroughputExceededException."); + } + return getShardIteratorResult.getShardIterator(); + } + + private List getShardsOfStream(String streamName, String lastSeenShardId) throws InterruptedException { + List shardsOfStream = new ArrayList<>(); + + DescribeStreamResult describeStreamResult; + do { + describeStreamResult = describeStream(streamName, lastSeenShardId); + + List shards = describeStreamResult.getStreamDescription().getShards(); + for (Shard shard : shards) { + shardsOfStream.add(new KinesisStreamShard(streamName, shard)); + } + + if (shards.size() != 0) { + lastSeenShardId = shards.get(shards.size() - 1).getShardId(); + } + } while (describeStreamResult.getStreamDescription().isHasMoreShards()); + + return shardsOfStream; } /** * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess. * + * This method is using a "full jitter" approach described in AWS's article, + * "Exponential Backoff and Jitter". + * This is necessary because concurrent calls will be made by all parallel subtask's fetcher. This + * jitter backoff approach will help distribute calls across the fetchers over time. + * * @param streamName the stream to describe * @param startShardId which shard to start with for this describe operation (earlier shard's infos will not appear in result) * @return the result of the describe stream operation */ - private DescribeStreamResult describeStream(String streamName, String startShardId) { + private DescribeStreamResult describeStream(String streamName, String startShardId) throws InterruptedException { final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest(); describeStreamRequest.setStreamName(streamName); describeStreamRequest.setExclusiveStartShardId(startShardId); DescribeStreamResult describeStreamResult = null; - String streamStatus = null; - int remainingRetryTimes = Integer.valueOf( - configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES))); - long describeStreamBackoffTimeInMillis = Long.valueOf( - configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF))); - - // Call DescribeStream, with backoff and retries (if we get LimitExceededException). - while ((remainingRetryTimes >= 0) && (describeStreamResult == null)) { + + // Call DescribeStream, with full-jitter backoff (if we get LimitExceededException). + int attemptCount = 0; + while (describeStreamResult == null) { // retry until we get a result try { describeStreamResult = kinesisClient.describeStream(describeStreamRequest); - streamStatus = describeStreamResult.getStreamDescription().getStreamStatus(); } catch (LimitExceededException le) { + long backoffMillis = fullJitterBackoff( + describeStreamBaseBackoffMillis, describeStreamMaxBackoffMillis, describeStreamExpConstant, attemptCount++); LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for " - + describeStreamBackoffTimeInMillis + " millis."); - try { - Thread.sleep(describeStreamBackoffTimeInMillis); - } catch (InterruptedException ie) { - LOG.debug("Stream " + streamName + " : Sleep was interrupted ", ie); - } + + backoffMillis + " millis."); + Thread.sleep(backoffMillis); } catch (ResourceNotFoundException re) { throw new RuntimeException("Error while getting stream details", re); } - remainingRetryTimes--; } - if (streamStatus == null) { - throw new RuntimeException("Can't get stream info from after 3 retries due to LimitExceededException"); - } else if (streamStatus.equals(StreamStatus.ACTIVE.toString()) || - streamStatus.equals(StreamStatus.UPDATING.toString())) { - return describeStreamResult; - } else { - throw new RuntimeException("Stream is not Active or Updating"); + String streamStatus = describeStreamResult.getStreamDescription().getStreamStatus(); + if (!(streamStatus.equals(StreamStatus.ACTIVE.toString()) || streamStatus.equals(StreamStatus.UPDATING.toString()))) { + if (LOG.isWarnEnabled()) { + LOG.warn("The status of stream " + streamName + " is " + streamStatus + "; result of the current " + + "describeStream operation will not contain any shard information."); + } } + + return describeStreamResult; + } + + private static long fullJitterBackoff(long base, long max, double power, int attempt) { + long exponentialBackoff = (long) Math.min(max, base * Math.pow(power, attempt)); + return (long)(seed.nextDouble()*exponentialBackoff); // random jitter between 0 and the exponential backoff } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java new file mode 100644 index 0000000000000..39ddc5233b48f --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.proxy; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; + +import java.util.Map; + +/** + * Interface for a Kinesis proxy that operates on multiple Kinesis streams within the same AWS service region. + */ +public interface KinesisProxyInterface { + + /** + * Get a shard iterator from the specified position in a shard. + * The retrieved shard iterator can be used in {@link KinesisProxyInterface#getRecords(String, int)}} + * to read data from the Kinesis shard. + * + * @param shard the shard to get the iterator + * @param shardIteratorType the iterator type, defining how the shard is to be iterated + * (one of: TRIM_HORIZON, LATEST, AT_SEQUENCE_NUMBER, AFTER_SEQUENCE_NUMBER) + * @param startingSeqNum sequence number, must be null if shardIteratorType is TRIM_HORIZON or LATEST + * @return shard iterator which can be used to read data from Kinesis + * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the + * operation has exceeded the rate limit; this exception will be thrown + * if the backoff is interrupted. + */ + String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) throws InterruptedException; + + /** + * Get the next batch of data records using a specific shard iterator + * + * @param shardIterator a shard iterator that encodes info about which shard to read and where to start reading + * @param maxRecordsToGet the maximum amount of records to retrieve for this batch + * @return the batch of retrieved records, also with a shard iterator that can be used to get the next batch + * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the + * operation has exceeded the rate limit; this exception will be thrown + * if the backoff is interrupted. + */ + GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException; + + /** + * Get shard list of multiple Kinesis streams, ignoring the + * shards of each stream before a specified last seen shard id. + * + * @param streamNamesWithLastSeenShardIds a map with stream as key, and last seen shard id as value + * @return result of the shard list query + * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the + * operation has exceeded the rate limit; this exception will be thrown + * if the backoff is interrupted. + */ + GetShardListResult getShardList(Map streamNamesWithLastSeenShardIds) throws InterruptedException; +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java index 187f0984f47bb..2eec0a4cddf5d 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java @@ -17,13 +17,18 @@ package org.apache.flink.streaming.connectors.kinesis.util; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.ClientConfigurationFactory; import com.amazonaws.auth.AWSCredentials; import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.auth.EnvironmentVariableCredentialsProvider; import com.amazonaws.auth.SystemPropertiesCredentialsProvider; import com.amazonaws.auth.profile.ProfileCredentialsProvider; +import com.amazonaws.regions.Region; import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; import org.apache.flink.streaming.connectors.kinesis.config.CredentialProviderType; @@ -34,6 +39,23 @@ */ public class AWSUtil { + /** + * Creates an Amazon Kinesis Client. + * @param configProps configuration properties containing the access key, secret key, and region + * @return a new Amazon Kinesis Client + */ + public static AmazonKinesisClient createKinesisClient(Properties configProps) { + // set a Flink-specific user agent + ClientConfiguration awsClientConfig = new ClientConfigurationFactory().getConfig(); + awsClientConfig.setUserAgent("Apache Flink " + EnvironmentInformation.getVersion() + + " (" + EnvironmentInformation.getRevisionInformation().commitId + ") Kinesis Connector"); + + AmazonKinesisClient client = + new AmazonKinesisClient(AWSUtil.getCredentialsProvider(configProps).getCredentials(), awsClientConfig); + client.setRegion(Region.getRegion(Regions.fromName(configProps.getProperty(KinesisConfigConstants.CONFIG_AWS_REGION)))); + return client; + } + /** * Return a {@link AWSCredentialsProvider} instance corresponding to the configuration properties. * diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java index 042b168f10f27..8e442de7ca45e 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java @@ -21,6 +21,7 @@ import org.apache.flink.streaming.connectors.kinesis.config.CredentialProviderType; import org.apache.flink.streaming.connectors.kinesis.config.InitialPosition; import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; import java.util.Properties; @@ -98,36 +99,98 @@ public static void validateConfiguration(Properties config) { } } - validateOptionalIntProperty(config, KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, - "Invalid value given for describeStream stream operation retry count. Must be a valid integer value."); + validateOptionalPositiveIntProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_MAX, + "Invalid value given for maximum records per getRecords shard operation. Must be a valid non-negative integer value."); - validateOptionalIntProperty(config, KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET, - "Invalid value given for maximum records per getRecords shard operation. Must be a valid integer value."); + validateOptionalPositiveIntProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_RETRIES, + "Invalid value given for maximum retry attempts for getRecords shard operation. Must be a valid non-negative integer value."); - validateOptionalLongProperty(config, KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, - "Invalid value given for describeStream stream operation backoff milliseconds. Must be a valid long value."); + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_BASE, + "Invalid value given for get records operation base backoff milliseconds. Must be a valid non-negative long value"); - validateOptionalLongProperty(config, KinesisConfigConstants.CONFIG_PRODUCER_COLLECTION_MAX_COUNT, - "Invalid value given for maximum number of items to pack into a PutRecords request. Must be a valid long value."); + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_MAX, + "Invalid value given for get records operation max backoff milliseconds. Must be a valid non-negative long value"); - validateOptionalLongProperty(config, KinesisConfigConstants.CONFIG_PRODUCER_AGGREGATION_MAX_COUNT, - "Invalid value given for maximum number of items to pack into an aggregated record. Must be a valid long value."); + validateOptionalPositiveDoubleProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT, + "Invalid value given for get records operation backoff exponential constant. Must be a valid non-negative double value"); + + validateOptionalPositiveIntProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_RETRIES, + "Invalid value given for maximum retry attempts for getShardIterator shard operation. Must be a valid non-negative integer value."); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_BASE, + "Invalid value given for get shard iterator operation base backoff milliseconds. Must be a valid non-negative long value"); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_MAX, + "Invalid value given for get shard iterator operation max backoff milliseconds. Must be a valid non-negative long value"); + + validateOptionalPositiveDoubleProperty(config, KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT, + "Invalid value given for get shard iterator operation backoff exponential constant. Must be a valid non-negative double value"); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS, + "Invalid value given for shard discovery sleep interval in milliseconds. Must be a valid non-negative long value"); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_BASE, + "Invalid value given for describe stream operation base backoff milliseconds. Must be a valid non-negative long value"); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_MAX, + "Invalid value given for describe stream operation max backoff milliseconds. Must be a valid non-negative long value"); + + validateOptionalPositiveDoubleProperty(config, KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + "Invalid value given for describe stream operation backoff exponential constant. Must be a valid non-negative double value"); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_PRODUCER_COLLECTION_MAX_COUNT, + "Invalid value given for maximum number of items to pack into a PutRecords request. Must be a valid non-negative long value."); + + validateOptionalPositiveLongProperty(config, KinesisConfigConstants.CONFIG_PRODUCER_AGGREGATION_MAX_COUNT, + "Invalid value given for maximum number of items to pack into an aggregated record. Must be a valid non-negative long value."); } - private static void validateOptionalLongProperty(Properties config, String key, String message) { + public static SentinelSequenceNumber getInitialPositionAsSentinelSequenceNumber(Properties config) { + InitialPosition initialPosition = InitialPosition.valueOf(config.getProperty( + KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, InitialPosition.LATEST.toString())); + + switch (initialPosition) { + case TRIM_HORIZON: + return SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM; + case LATEST: + default: + return SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM; + } + } + + private static void validateOptionalPositiveLongProperty(Properties config, String key, String message) { + if (config.containsKey(key)) { + try { + long value = Long.parseLong(config.getProperty(key)); + if (value < 0) { + throw new NumberFormatException(); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException(message); + } + } + } + + private static void validateOptionalPositiveIntProperty(Properties config, String key, String message) { if (config.containsKey(key)) { try { - Long.parseLong(config.getProperty(key)); + int value = Integer.parseInt(config.getProperty(key)); + if (value < 0) { + throw new NumberFormatException(); + } } catch (NumberFormatException e) { throw new IllegalArgumentException(message); } } } - private static void validateOptionalIntProperty(Properties config, String key, String message) { + private static void validateOptionalPositiveDoubleProperty(Properties config, String key, String message) { if (config.containsKey(key)) { try { - Integer.parseInt(config.getProperty(key)); + double value = Double.parseDouble(config.getProperty(key)); + if (value < 0) { + throw new NumberFormatException(); + } } catch (NumberFormatException e) { throw new IllegalArgumentException(message); } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java index 5ced01964f107..ec9ee9a207f43 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java @@ -17,16 +17,18 @@ package org.apache.flink.streaming.connectors.kinesis; +import com.amazonaws.services.kinesis.model.Shard; import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; -import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; -import org.apache.flink.streaming.connectors.kinesis.testutils.ReferenceKinesisShardTopologies; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; import org.apache.flink.streaming.connectors.kinesis.testutils.TestableFlinkKinesisConsumer; import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -36,26 +38,15 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.UUID; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -import static org.powermock.api.mockito.PowerMockito.mock; -import static org.powermock.api.mockito.PowerMockito.mockStatic; -import static org.powermock.api.mockito.PowerMockito.whenNew; /** - * Suite of FlinkKinesisConsumer tests, including utility static method tests, - * and tests for the methods called throughout the source life cycle with mocked KinesisProxy. + * Suite of FlinkKinesisConsumer tests for the methods called throughout the source life cycle. */ @RunWith(PowerMockRunner.class) @PrepareForTest({FlinkKinesisConsumer.class, KinesisConfigUtil.class}) @@ -148,365 +139,282 @@ public void testUnrecognizableStreamInitPositionTypeInConfig() { } @Test - public void testUnparsableIntForDescribeStreamRetryCountInConfig() { + public void testUnparsableLongForDescribeStreamBackoffBaseMillisInConfig() { exception.expect(IllegalArgumentException.class); - exception.expectMessage("Invalid value given for describeStream stream operation retry count"); + exception.expectMessage("Invalid value given for describe stream operation base backoff milliseconds"); Properties testConfig = new Properties(); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, "unparsableInt"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_BASE, "unparsableLong"); KinesisConfigUtil.validateConfiguration(testConfig); } @Test - public void testUnparsableLongForDescribeStreamBackoffMillisInConfig() { + public void testUnparsableLongForDescribeStreamBackoffMaxMillisInConfig() { exception.expect(IllegalArgumentException.class); - exception.expectMessage("Invalid value given for describeStream stream operation backoff milliseconds"); + exception.expectMessage("Invalid value given for describe stream operation max backoff milliseconds"); Properties testConfig = new Properties(); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, "unparsableLong"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_MAX, "unparsableLong"); KinesisConfigUtil.validateConfiguration(testConfig); } @Test - public void testUnparsableIntForGetRecordsMaxCountInConfig() { + public void testUnparsableDoubleForDescribeStreamBackoffExponentialConstantInConfig() { exception.expect(IllegalArgumentException.class); - exception.expectMessage("Invalid value given for maximum records per getRecords shard operation"); + exception.expectMessage("Invalid value given for describe stream operation backoff exponential constant"); Properties testConfig = new Properties(); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_RECORDS_PER_GET, "unparsableInt"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, "unparsableDouble"); KinesisConfigUtil.validateConfiguration(testConfig); } - // ---------------------------------------------------------------------- - // FlinkKinesisConsumer.assignShards() tests - // ---------------------------------------------------------------------- - @Test - public void testShardNumEqualConsumerNum() { - try { - List fakeShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - int consumerTaskCount = fakeShards.size(); - - for (int consumerNum=0; consumerNum < consumerTaskCount; consumerNum++) { - List assignedShardsToThisConsumerTask = - FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); - - // the ith consumer should be assigned exactly 1 shard, - // which is always the ith shard of a shard list that only has open shards - assertEquals(1, assignedShardsToThisConsumerTask.size()); - assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(consumerNum))); - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + public void testUnparsableIntForGetRecordsRetriesInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for maximum retry attempts for getRecords shard operation"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_RETRIES, "unparsableInt"); + + KinesisConfigUtil.validateConfiguration(testConfig); } @Test - public void testShardNumFewerThanConsumerNum() { - try { - List fakeShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - int consumerTaskCount = fakeShards.size() + 3; - - for (int consumerNum = 0; consumerNum < consumerTaskCount; consumerNum++) { - List assignedShardsToThisConsumerTask = - FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); - - // for ith consumer with i < the total num of shards, - // the ith consumer should be assigned exactly 1 shard, - // which is always the ith shard of a shard list that only has open shards; - // otherwise, the consumer should not be assigned any shards - if (consumerNum < fakeShards.size()) { - assertEquals(1, assignedShardsToThisConsumerTask.size()); - assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(consumerNum))); - } else { - assertEquals(0, assignedShardsToThisConsumerTask.size()); - } - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + public void testUnparsableIntForGetRecordsMaxCountInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for maximum records per getRecords shard operation"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_MAX, "unparsableInt"); + + KinesisConfigUtil.validateConfiguration(testConfig); } @Test - public void testShardNumMoreThanConsumerNum() { - try { - List fakeShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - int consumerTaskCount = fakeShards.size() - 1; - - for (int consumerNum = 0; consumerNum < consumerTaskCount; consumerNum++) { - List assignedShardsToThisConsumerTask = - FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); - - // since the number of consumer tasks is short by 1, - // all but the first consumer task should be assigned 1 shard, - // while the first consumer task is assigned 2 shards - if (consumerNum != 0) { - assertEquals(1, assignedShardsToThisConsumerTask.size()); - assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(consumerNum))); - } else { - assertEquals(2, assignedShardsToThisConsumerTask.size()); - assertTrue(assignedShardsToThisConsumerTask.get(0).equals(fakeShards.get(0))); - assertTrue(assignedShardsToThisConsumerTask.get(1).equals(fakeShards.get(fakeShards.size()-1))); - } - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + public void testUnparsableLongForGetRecordsBackoffBaseMillisInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for get records operation base backoff milliseconds"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_BASE, "unparsableLong"); + + KinesisConfigUtil.validateConfiguration(testConfig); } @Test - public void testAssignEmptyShards() { - try { - List fakeShards = new ArrayList<>(0); - int consumerTaskCount = 4; - - for (int consumerNum = 0; consumerNum < consumerTaskCount; consumerNum++) { - List assignedShardsToThisConsumerTask = - FlinkKinesisConsumer.assignShards(fakeShards, consumerTaskCount, consumerNum); - - // should not be assigned anything - assertEquals(0, assignedShardsToThisConsumerTask.size()); - - } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } + public void testUnparsableLongForGetRecordsBackoffMaxMillisInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for get records operation max backoff milliseconds"); - // ---------------------------------------------------------------------- - // Constructor tests with mocked KinesisProxy - // ---------------------------------------------------------------------- + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_MAX, "unparsableLong"); + + KinesisConfigUtil.validateConfiguration(testConfig); + } @Test - public void testConstructorShouldThrowRuntimeExceptionIfUnableToFindAnyShards() { - exception.expect(RuntimeException.class); - exception.expectMessage("Unable to retrieve any shards"); - - Properties testConsumerConfig = new Properties(); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - - // get a consumer that will not be able to find any shards from AWS Kinesis - FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( - 6, 2, "fake-consumer-task-name", - new ArrayList(), new ArrayList(), testConsumerConfig, - null, null, false, false); + public void testUnparsableDoubleForGetRecordsBackoffExponentialConstantInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for get records operation backoff exponential constant"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT, "unparsableDouble"); + + KinesisConfigUtil.validateConfiguration(testConfig); } - // ---------------------------------------------------------------------- - // Tests for open() source life cycle method - // ---------------------------------------------------------------------- + @Test + public void testUnparsableIntForGetShardIteratorRetriesInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for maximum retry attempts for getShardIterator shard operation"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_RETRIES, "unparsableInt"); + + KinesisConfigUtil.validateConfiguration(testConfig); + } @Test - public void testOpenWithNoRestoreStateFetcherAdvanceToLatestSentinelSequenceNumberWhenConfigSetToStartFromLatest() throws Exception { - - int fakeNumConsumerTasks = 6; - int fakeThisConsumerTaskIndex = 2; - String fakeThisConsumerTaskName = "fake-this-task-name"; - - List fakeCompleteShardList = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - List fakeAssignedShardsToThisConsumerTask = fakeCompleteShardList.subList(2,3); - - Properties testConsumerConfig = new Properties(); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "LATEST"); - - KinesisDataFetcher kinesisDataFetcherMock = Mockito.mock(KinesisDataFetcher.class); - try { - whenNew(KinesisDataFetcher.class).withArguments(fakeAssignedShardsToThisConsumerTask, testConsumerConfig, fakeThisConsumerTaskName).thenReturn(kinesisDataFetcherMock); - } catch (Exception e) { - throw new RuntimeException("Error when power mocking KinesisDataFetcher in test", e); - } + public void testUnparsableLongForGetShardIteratorBackoffBaseMillisInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for get shard iterator operation base backoff milliseconds"); - FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( - fakeNumConsumerTasks, fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, - fakeCompleteShardList, fakeAssignedShardsToThisConsumerTask, testConsumerConfig, - null, null, false, false); + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_BASE, "unparsableLong"); - dummyConsumer.open(new Configuration()); + KinesisConfigUtil.validateConfiguration(testConfig); + } - for (KinesisStreamShard shard : fakeAssignedShardsToThisConsumerTask) { - verify(kinesisDataFetcherMock).advanceSequenceNumberTo(shard, SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get()); - } + @Test + public void testUnparsableLongForGetShardIteratorBackoffMaxMillisInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for get shard iterator operation max backoff milliseconds"); + + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_MAX, "unparsableLong"); + KinesisConfigUtil.validateConfiguration(testConfig); } @Test - public void testOpenWithNoRestoreStateFetcherAdvanceToEarliestSentinelSequenceNumberWhenConfigSetToTrimHorizon() throws Exception { - - int fakeNumConsumerTasks = 6; - int fakeThisConsumerTaskIndex = 2; - String fakeThisConsumerTaskName = "fake-this-task-name"; - - List fakeCompleteShardList = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - List fakeAssignedShardsToThisConsumerTask = fakeCompleteShardList.subList(2,3); - - Properties testConsumerConfig = new Properties(); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "TRIM_HORIZON"); - - KinesisDataFetcher kinesisDataFetcherMock = Mockito.mock(KinesisDataFetcher.class); - try { - whenNew(KinesisDataFetcher.class).withArguments(fakeAssignedShardsToThisConsumerTask, testConsumerConfig, fakeThisConsumerTaskName).thenReturn(kinesisDataFetcherMock); - } catch (Exception e) { - throw new RuntimeException("Error when power mocking KinesisDataFetcher in test", e); - } + public void testUnparsableDoubleForGetShardIteratorBackoffExponentialConstantInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for get shard iterator operation backoff exponential constant"); - FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( - fakeNumConsumerTasks, fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, - fakeCompleteShardList, fakeAssignedShardsToThisConsumerTask, testConsumerConfig, - null, null, false, false); + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT, "unparsableDouble"); - dummyConsumer.open(new Configuration()); + KinesisConfigUtil.validateConfiguration(testConfig); + } - for (KinesisStreamShard shard : fakeAssignedShardsToThisConsumerTask) { - verify(kinesisDataFetcherMock).advanceSequenceNumberTo(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get()); - } + @Test + public void testUnparsableLongForShardDiscoveryIntervalMillisInConfig() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Invalid value given for shard discovery sleep interval in milliseconds"); + Properties testConfig = new Properties(); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); + testConfig.setProperty(KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS, "unparsableLong"); + + KinesisConfigUtil.validateConfiguration(testConfig); } + // ---------------------------------------------------------------------- + // Tests related to state initialization + // ---------------------------------------------------------------------- + @Test - public void testOpenWithRestoreStateFetcherAdvanceToCorrespondingSequenceNumbers() throws Exception { - - int fakeNumConsumerTasks = 6; - int fakeThisConsumerTaskIndex = 2; - String fakeThisConsumerTaskName = "fake-this-task-name"; - - List fakeCompleteShardList = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - List fakeAssignedShardsToThisConsumerTask = fakeCompleteShardList.subList(2,3); - - Properties testConsumerConfig = new Properties(); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - testConsumerConfig.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "TRIM_HORIZON"); - - KinesisDataFetcher kinesisDataFetcherMock = Mockito.mock(KinesisDataFetcher.class); - try { - whenNew(KinesisDataFetcher.class).withArguments(fakeAssignedShardsToThisConsumerTask, testConsumerConfig, fakeThisConsumerTaskName).thenReturn(kinesisDataFetcherMock); - } catch (Exception e) { - throw new RuntimeException("Error when power mocking KinesisDataFetcher in test", e); - } + public void testSnapshotStateShouldBeNullIfSourceNotOpened() throws Exception { + Properties config = new Properties(); + config.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1"); + config.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, "accessKeyId"); + config.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, "secretKey"); - FlinkKinesisConsumer dummyConsumer = getDummyConsumerWithMockedKinesisProxy( - fakeNumConsumerTasks, fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, - fakeCompleteShardList, fakeAssignedShardsToThisConsumerTask, testConsumerConfig, - null, null, false, false); + FlinkKinesisConsumer consumer = new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config); - // generate random UUIDs as sequence numbers of last checkpointed state for each assigned shard - ArrayList listOfSeqNumOfAssignedShards = new ArrayList<>(fakeAssignedShardsToThisConsumerTask.size()); - for (KinesisStreamShard shard : fakeAssignedShardsToThisConsumerTask) { - listOfSeqNumOfAssignedShards.add(new SequenceNumber(UUID.randomUUID().toString())); - } + assertTrue(consumer.snapshotState(123, 123) == null); //arbitrary checkpoint id and timestamp + } - HashMap fakeRestoredState = new HashMap<>(); - for (int i=0; i consumer = new FlinkKinesisConsumer<>("fakeStream", new SimpleStringSchema(), config); + consumer.open(new Configuration()); // only opened, not run - for (int i=0; i fakeCompleteShardList, - List fakeAssignedShardListToThisConsumerTask, - Properties consumerTestConfig, - KinesisDataFetcher fetcher, - HashMap lastSequenceNumsToRestore, - boolean hasAssignedShards, - boolean running) { - - final String dummyKinesisStreamName = "flink-test"; - - final List dummyKinesisStreamList = Collections.singletonList(dummyKinesisStreamName); - - final KinesisProxy kinesisProxyMock = mock(KinesisProxy.class); - - // mock KinesisProxy that is instantiated in the constructor, as well as its getShardList call - try { - whenNew(KinesisProxy.class).withArguments(consumerTestConfig).thenReturn(kinesisProxyMock); - } catch (Exception e) { - throw new RuntimeException("Error when power mocking KinesisProxy in tests", e); - } + // ---------------------------------------------------------------------- + // Tests related to fetcher initialization + // ---------------------------------------------------------------------- - when(kinesisProxyMock.getShardList(dummyKinesisStreamList)).thenReturn(fakeCompleteShardList); + @Test + @SuppressWarnings("unchecked") + public void testFetcherShouldNotBeRestoringFromFailureIfNotRestoringFromCheckpoint() throws Exception { + KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class); + PowerMockito.whenNew(KinesisDataFetcher.class).withAnyArguments().thenReturn(mockedFetcher); - TestableFlinkKinesisConsumer dummyConsumer = - new TestableFlinkKinesisConsumer(dummyKinesisStreamName, fakeNumFlinkConsumerTasks, - fakeThisConsumerTaskIndex, fakeThisConsumerTaskName, consumerTestConfig); + // assume the given config is correct + PowerMockito.mockStatic(KinesisConfigUtil.class); + PowerMockito.doNothing().when(KinesisConfigUtil.class); - try { - Field fetcherField = FlinkKinesisConsumer.class.getDeclaredField("fetcher"); - fetcherField.setAccessible(true); - fetcherField.set(dummyConsumer, fetcher); + TestableFlinkKinesisConsumer consumer = new TestableFlinkKinesisConsumer( + "fakeStream", new Properties(), 10, 2); + consumer.open(new Configuration()); + consumer.run(Mockito.mock(SourceFunction.SourceContext.class)); - Field lastSequenceNumsField = FlinkKinesisConsumer.class.getDeclaredField("lastSequenceNums"); - lastSequenceNumsField.setAccessible(true); - lastSequenceNumsField.set(dummyConsumer, lastSequenceNumsToRestore); + Mockito.verify(mockedFetcher).setIsRestoringFromFailure(false); + } - Field hasAssignedShardsField = FlinkKinesisConsumer.class.getDeclaredField("hasAssignedShards"); - hasAssignedShardsField.setAccessible(true); - hasAssignedShardsField.set(dummyConsumer, hasAssignedShards); + @Test + @SuppressWarnings("unchecked") + public void testFetcherShouldBeCorrectlySeededIfRestoringFromCheckpoint() throws Exception { + KinesisDataFetcher mockedFetcher = Mockito.mock(KinesisDataFetcher.class); + PowerMockito.whenNew(KinesisDataFetcher.class).withAnyArguments().thenReturn(mockedFetcher); - Field runningField = FlinkKinesisConsumer.class.getDeclaredField("running"); - runningField.setAccessible(true); - runningField.set(dummyConsumer, running); - } catch (IllegalAccessException | NoSuchFieldException e) { - // no reason to end up here ... - throw new RuntimeException(e); - } + // assume the given config is correct + PowerMockito.mockStatic(KinesisConfigUtil.class); + PowerMockito.doNothing().when(KinesisConfigUtil.class); - // mock FlinkKinesisConsumer utility static methods - mockStatic(FlinkKinesisConsumer.class); - mockStatic(KinesisConfigUtil.class); - - try { - // assume assignShards static method is correct by mocking - PowerMockito.when( - FlinkKinesisConsumer.assignShards( - fakeCompleteShardList, - fakeNumFlinkConsumerTasks, - fakeThisConsumerTaskIndex)) - .thenReturn(fakeAssignedShardListToThisConsumerTask); - - // assume validatePropertiesConfig static method is correct by mocking - PowerMockito.doNothing().when(KinesisConfigUtil.class, "validateConfiguration", Mockito.any(Properties.class)); - } catch (Exception e) { - e.printStackTrace(); - throw new RuntimeException("Error when power mocking static methods of FlinkKinesisConsumer", e); + HashMap fakeRestoredState = new HashMap<>(); + fakeRestoredState.put( + new KinesisStreamShard("fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + new SequenceNumber(UUID.randomUUID().toString())); + fakeRestoredState.put( + new KinesisStreamShard("fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + new SequenceNumber(UUID.randomUUID().toString())); + fakeRestoredState.put( + new KinesisStreamShard("fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))), + new SequenceNumber(UUID.randomUUID().toString())); + fakeRestoredState.put( + new KinesisStreamShard("fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + new SequenceNumber(UUID.randomUUID().toString())); + fakeRestoredState.put( + new KinesisStreamShard("fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + new SequenceNumber(UUID.randomUUID().toString())); + + TestableFlinkKinesisConsumer consumer = new TestableFlinkKinesisConsumer( + "fakeStream", new Properties(), 10, 2); + consumer.restoreState(fakeRestoredState); + consumer.open(new Configuration()); + consumer.run(Mockito.mock(SourceFunction.SourceContext.class)); + + Mockito.verify(mockedFetcher).setIsRestoringFromFailure(true); + for (Map.Entry restoredShard : fakeRestoredState.entrySet()) { + Mockito.verify(mockedFetcher).advanceLastDiscoveredShardOfStream( + restoredShard.getKey().getStreamName(), restoredShard.getKey().getShard().getShardId()); + Mockito.verify(mockedFetcher).registerNewSubscribedShardState( + new KinesisStreamShardState(restoredShard.getKey(), restoredShard.getValue())); } - - return dummyConsumer; } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java index e099a5c440a6b..e79f9b14628b6 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcherTest.java @@ -17,33 +17,494 @@ package org.apache.flink.streaming.connectors.kinesis.internals; +import com.amazonaws.services.kinesis.model.Shard; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; -import org.apache.flink.streaming.connectors.kinesis.testutils.ReferenceKinesisShardTopologies; -import org.junit.Rule; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; +import org.apache.flink.streaming.connectors.kinesis.testutils.FakeKinesisBehavioursFactory; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; +import org.apache.flink.streaming.connectors.kinesis.testutils.TestableKinesisDataFetcher; import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import java.util.HashMap; +import java.util.Map; +import java.util.LinkedList; import java.util.List; import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import static org.junit.Assert.assertTrue; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(TestableKinesisDataFetcher.class) public class KinesisDataFetcherTest { - @Rule - public ExpectedException exception = ExpectedException.none(); + @Test(expected = RuntimeException.class) + public void testIfNoShardsAreFoundShouldThrowException() throws Exception { + List fakeStreams = new LinkedList<>(); + fakeStreams.add("fakeStream1"); + fakeStreams.add("fakeStream2"); + + HashMap subscribedStreamsToLastSeenShardIdsUnderTest = + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams); + + TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + fakeStreams, + new Properties(), + 10, + 2, + new AtomicReference(), + new LinkedList(), + subscribedStreamsToLastSeenShardIdsUnderTest, + FakeKinesisBehavioursFactory.noShardsFoundForRequestedStreamsBehaviour()); + + fetcher.setIsRestoringFromFailure(false); // not restoring + + fetcher.runFetcher(); // this should throw RuntimeException + } + + @Test + public void testStreamToLastSeenShardStateIsCorrectlySetWhenNotRestoringFromFailure() throws Exception { + List fakeStreams = new LinkedList<>(); + fakeStreams.add("fakeStream1"); + fakeStreams.add("fakeStream2"); + fakeStreams.add("fakeStream3"); + fakeStreams.add("fakeStream4"); + + HashMap subscribedStreamsToLastSeenShardIdsUnderTest = + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams); + + Map streamToShardCount = new HashMap<>(); + Random rand = new Random(); + for (String fakeStream : fakeStreams) { + streamToShardCount.put(fakeStream, rand.nextInt(5)+1); + } + + final TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + fakeStreams, + new Properties(), + 10, + 2, + new AtomicReference(), + new LinkedList(), + subscribedStreamsToLastSeenShardIdsUnderTest, + FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount)); + + fetcher.setIsRestoringFromFailure(false); + + PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class)); + Thread runFetcherThread = new Thread(new Runnable() { + @Override + public void run() { + try { + fetcher.runFetcher(); + } catch (Exception e) { + // + } + } + }); + runFetcherThread.start(); + Thread.sleep(1000); // sleep a while before closing + fetcher.shutdownFetcher(); + + + // assert that the streams tracked in the state are identical to the subscribed streams + Set streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet(); + assertTrue(streamsInState.size() == fakeStreams.size()); + assertTrue(streamsInState.containsAll(fakeStreams)); + + // assert that the last seen shards in state is correctly set + for (Map.Entry streamToLastSeenShard : subscribedStreamsToLastSeenShardIdsUnderTest.entrySet()) { + assertTrue( + streamToLastSeenShard.getValue().equals( + KinesisShardIdGenerator.generateFromShardOrder(streamToShardCount.get(streamToLastSeenShard.getKey())-1))); + } + } + + @Test + public void testStreamToLastSeenShardStateIsCorrectlySetWhenNoNewShardsSinceRestoredCheckpoint() throws Exception { + List fakeStreams = new LinkedList<>(); + fakeStreams.add("fakeStream1"); + fakeStreams.add("fakeStream2"); + + Map restoredStateUnderTest = new HashMap<>(); + + // fakeStream1 has 3 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))), + UUID.randomUUID().toString()); + + // fakeStream2 has 2 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + + Map streamToShardCount = new HashMap<>(); + streamToShardCount.put("fakeStream1", 3); // fakeStream1 will still have 3 shards after restore + streamToShardCount.put("fakeStream2", 2); // fakeStream2 will still have 2 shards after restore + + HashMap subscribedStreamsToLastSeenShardIdsUnderTest = + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams); + + final TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + fakeStreams, + new Properties(), + 10, + 2, + new AtomicReference(), + new LinkedList(), + subscribedStreamsToLastSeenShardIdsUnderTest, + FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount)); + + for (Map.Entry restoredState : restoredStateUnderTest.entrySet()) { + fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId()); + fetcher.registerNewSubscribedShardState( + new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue()))); + } + + fetcher.setIsRestoringFromFailure(true); + + PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class)); + Thread runFetcherThread = new Thread(new Runnable() { + @Override + public void run() { + try { + fetcher.runFetcher(); + } catch (Exception e) { + // + } + } + }); + runFetcherThread.start(); + Thread.sleep(1000); // sleep a while before closing + fetcher.shutdownFetcher(); + + // assert that the streams tracked in the state are identical to the subscribed streams + Set streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet(); + assertTrue(streamsInState.size() == fakeStreams.size()); + assertTrue(streamsInState.containsAll(fakeStreams)); + + // assert that the last seen shards in state is correctly set + for (Map.Entry streamToLastSeenShard : subscribedStreamsToLastSeenShardIdsUnderTest.entrySet()) { + assertTrue( + streamToLastSeenShard.getValue().equals( + KinesisShardIdGenerator.generateFromShardOrder(streamToShardCount.get(streamToLastSeenShard.getKey())-1))); + } + } + + @Test + public void testStreamToLastSeenShardStateIsCorrectlySetWhenNewShardsFoundSinceRestoredCheckpoint() throws Exception { + List fakeStreams = new LinkedList<>(); + fakeStreams.add("fakeStream1"); + fakeStreams.add("fakeStream2"); + + Map restoredStateUnderTest = new HashMap<>(); + + // fakeStream1 has 3 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))), + UUID.randomUUID().toString()); + + // fakeStream2 has 2 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + + Map streamToShardCount = new HashMap<>(); + streamToShardCount.put("fakeStream1", 3+1); // fakeStream1 had 3 shards before & 1 new shard after restore + streamToShardCount.put("fakeStream2", 2+3); // fakeStream2 had 2 shards before & 3 new shard after restore + + HashMap subscribedStreamsToLastSeenShardIdsUnderTest = + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams); + + // using a non-resharded streams kinesis behaviour to represent that Kinesis is not resharded AFTER the restore + final TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + fakeStreams, + new Properties(), + 10, + 2, + new AtomicReference(), + new LinkedList(), + subscribedStreamsToLastSeenShardIdsUnderTest, + FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount)); + + for (Map.Entry restoredState : restoredStateUnderTest.entrySet()) { + fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId()); + fetcher.registerNewSubscribedShardState( + new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue()))); + } + + fetcher.setIsRestoringFromFailure(true); + + PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class)); + Thread runFetcherThread = new Thread(new Runnable() { + @Override + public void run() { + try { + fetcher.runFetcher(); + } catch (Exception e) { + // + } + } + }); + runFetcherThread.start(); + Thread.sleep(1000); // sleep a while before closing + fetcher.shutdownFetcher(); + + // assert that the streams tracked in the state are identical to the subscribed streams + Set streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet(); + assertTrue(streamsInState.size() == fakeStreams.size()); + assertTrue(streamsInState.containsAll(fakeStreams)); + + // assert that the last seen shards in state is correctly set + for (Map.Entry streamToLastSeenShard : subscribedStreamsToLastSeenShardIdsUnderTest.entrySet()) { + assertTrue( + streamToLastSeenShard.getValue().equals( + KinesisShardIdGenerator.generateFromShardOrder(streamToShardCount.get(streamToLastSeenShard.getKey())-1))); + } + } @Test - public void testAdvanceSequenceNumberOnNotOwnedShard() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Can't advance sequence number on a shard we are not going to read."); + public void testStreamToLastSeenShardStateIsCorrectlySetWhenNoNewShardsSinceRestoredCheckpointAndSomeStreamsDoNotExist() throws Exception { + List fakeStreams = new LinkedList<>(); + fakeStreams.add("fakeStream1"); + fakeStreams.add("fakeStream2"); + fakeStreams.add("fakeStream3"); // fakeStream3 will not have any shards + fakeStreams.add("fakeStream4"); // fakeStream4 will not have any shards + + Map restoredStateUnderTest = new HashMap<>(); + + // fakeStream1 has 3 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))), + UUID.randomUUID().toString()); + + // fakeStream2 has 2 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + + Map streamToShardCount = new HashMap<>(); + streamToShardCount.put("fakeStream1", 3); // fakeStream1 has fixed 3 shards + streamToShardCount.put("fakeStream2", 2); // fakeStream2 has fixed 2 shards + streamToShardCount.put("fakeStream3", 0); // no shards can be found for fakeStream3 + streamToShardCount.put("fakeStream4", 0); // no shards can be found for fakeStream4 + + HashMap subscribedStreamsToLastSeenShardIdsUnderTest = + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams); - List fakeCompleteListOfShards = ReferenceKinesisShardTopologies.flatTopologyWithFourOpenShards(); - List assignedShardsToThisFetcher = fakeCompleteListOfShards.subList(0,1); + // using a non-resharded streams kinesis behaviour to represent that Kinesis is not resharded AFTER the restore + final TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + fakeStreams, + new Properties(), + 10, + 2, + new AtomicReference(), + new LinkedList(), + subscribedStreamsToLastSeenShardIdsUnderTest, + FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount)); - KinesisDataFetcher fetcherUnderTest = new KinesisDataFetcher(assignedShardsToThisFetcher, new Properties(), "fake-task-name"); + for (Map.Entry restoredState : restoredStateUnderTest.entrySet()) { + fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId()); + fetcher.registerNewSubscribedShardState( + new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue()))); + } - // advance the fetcher on a shard that it does not own - fetcherUnderTest.advanceSequenceNumberTo(fakeCompleteListOfShards.get(2), new SequenceNumber("fake-seq-num")); + fetcher.setIsRestoringFromFailure(true); + + PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class)); + Thread runFetcherThread = new Thread(new Runnable() { + @Override + public void run() { + try { + fetcher.runFetcher(); + } catch (Exception e) { + // + } + } + }); + runFetcherThread.start(); + Thread.sleep(1000); // sleep a while before closing + fetcher.shutdownFetcher(); + + // assert that the streams tracked in the state are identical to the subscribed streams + Set streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet(); + assertTrue(streamsInState.size() == fakeStreams.size()); + assertTrue(streamsInState.containsAll(fakeStreams)); + + // assert that the last seen shards in state is correctly set + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream1").equals( + KinesisShardIdGenerator.generateFromShardOrder(2))); + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream2").equals( + KinesisShardIdGenerator.generateFromShardOrder(1))); + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream3") == null); + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream4") == null); } + @Test + public void testStreamToLastSeenShardStateIsCorrectlySetWhenNewShardsFoundSinceRestoredCheckpointAndSomeStreamsDoNotExist() throws Exception { + List fakeStreams = new LinkedList<>(); + fakeStreams.add("fakeStream1"); + fakeStreams.add("fakeStream2"); + fakeStreams.add("fakeStream3"); // fakeStream3 will not have any shards + fakeStreams.add("fakeStream4"); // fakeStream4 will not have any shards + + Map restoredStateUnderTest = new HashMap<>(); + + // fakeStream1 has 3 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream1", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(2))), + UUID.randomUUID().toString()); + + // fakeStream2 has 2 shards before restore + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(0))), + UUID.randomUUID().toString()); + restoredStateUnderTest.put( + new KinesisStreamShard( + "fakeStream2", + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(1))), + UUID.randomUUID().toString()); + + Map streamToShardCount = new HashMap<>(); + streamToShardCount.put("fakeStream1", 3+1); // fakeStream1 had 3 shards before & 1 new shard after restore + streamToShardCount.put("fakeStream2", 2+3); // fakeStream2 had 2 shards before & 2 new shard after restore + streamToShardCount.put("fakeStream3", 0); // no shards can be found for fakeStream3 + streamToShardCount.put("fakeStream4", 0); // no shards can be found for fakeStream4 + + HashMap subscribedStreamsToLastSeenShardIdsUnderTest = + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(fakeStreams); + + // using a non-resharded streams kinesis behaviour to represent that Kinesis is not resharded AFTER the restore + final TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + fakeStreams, + new Properties(), + 10, + 2, + new AtomicReference(), + new LinkedList(), + subscribedStreamsToLastSeenShardIdsUnderTest, + FakeKinesisBehavioursFactory.nonReshardedStreamsBehaviour(streamToShardCount)); + + for (Map.Entry restoredState : restoredStateUnderTest.entrySet()) { + fetcher.advanceLastDiscoveredShardOfStream(restoredState.getKey().getStreamName(), restoredState.getKey().getShard().getShardId()); + fetcher.registerNewSubscribedShardState( + new KinesisStreamShardState(restoredState.getKey(), new SequenceNumber(restoredState.getValue()))); + } + + fetcher.setIsRestoringFromFailure(true); + + PowerMockito.whenNew(ShardConsumer.class).withAnyArguments().thenReturn(Mockito.mock(ShardConsumer.class)); + Thread runFetcherThread = new Thread(new Runnable() { + @Override + public void run() { + try { + fetcher.runFetcher(); + } catch (Exception e) { + // + } + } + }); + runFetcherThread.start(); + Thread.sleep(1000); // sleep a while before closing + fetcher.shutdownFetcher(); + + // assert that the streams tracked in the state are identical to the subscribed streams + Set streamsInState = subscribedStreamsToLastSeenShardIdsUnderTest.keySet(); + assertTrue(streamsInState.size() == fakeStreams.size()); + assertTrue(streamsInState.containsAll(fakeStreams)); + + // assert that the last seen shards in state is correctly set + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream1").equals( + KinesisShardIdGenerator.generateFromShardOrder(3))); + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream2").equals( + KinesisShardIdGenerator.generateFromShardOrder(4))); + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream3") == null); + assertTrue(subscribedStreamsToLastSeenShardIdsUnderTest.get("fakeStream4") == null); + } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java new file mode 100644 index 0000000000000..5b3e1a5666e37 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerTest.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.internals; + +import com.amazonaws.services.kinesis.model.HashKeyRange; +import com.amazonaws.services.kinesis.model.Shard; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; +import org.apache.flink.streaming.connectors.kinesis.testutils.FakeKinesisBehavioursFactory; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; +import org.apache.flink.streaming.connectors.kinesis.testutils.TestableKinesisDataFetcher; +import org.junit.Test; +import org.mockito.Mockito; + +import java.math.BigInteger; +import java.util.Collections; +import java.util.LinkedList; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertTrue; + +public class ShardConsumerTest { + + @Test + public void testCorrectNumOfCollectedRecordsAndUpdatedState() { + KinesisStreamShard fakeToBeConsumedShard = new KinesisStreamShard( + "fakeStream", + new Shard() + .withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)) + .withHashKeyRange( + new HashKeyRange() + .withStartingHashKey("0") + .withEndingHashKey(new BigInteger(StringUtils.repeat("FF", 16), 16).toString()))); + + LinkedList subscribedShardsStateUnderTest = new LinkedList<>(); + subscribedShardsStateUnderTest.add( + new KinesisStreamShardState(fakeToBeConsumedShard, new SequenceNumber("fakeStartingState"))); + + TestableKinesisDataFetcher fetcher = + new TestableKinesisDataFetcher( + Collections.singletonList("fakeStream"), + new Properties(), + 10, + 2, + new AtomicReference(), + subscribedShardsStateUnderTest, + KinesisDataFetcher.createInitialSubscribedStreamsToLastDiscoveredShardsState(Collections.singletonList("fakeStream")), + Mockito.mock(KinesisProxyInterface.class)); + + new ShardConsumer<>( + fetcher, + 0, + subscribedShardsStateUnderTest.get(0).getKinesisStreamShard(), + subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum(), + FakeKinesisBehavioursFactory.totalNumOfRecordsAfterNumOfGetRecordsCalls(1000, 9)).run(); + + assertTrue(fetcher.getNumOfElementsCollected() == 1000); + assertTrue(subscribedShardsStateUnderTest.get(0).getLastProcessedSequenceNum().equals( + SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())); + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java deleted file mode 100644 index 38937ecfb10c1..0000000000000 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumerThreadTest.java +++ /dev/null @@ -1,222 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.internals; - -import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; - -import com.amazonaws.services.kinesis.model.GetRecordsResult; -import com.amazonaws.services.kinesis.model.HashKeyRange; -import com.amazonaws.services.kinesis.model.Record; -import com.amazonaws.services.kinesis.model.Shard; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; -import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; -import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; -import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper; -import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Matchers; -import org.mockito.Mockito; -import org.powermock.api.mockito.PowerMockito; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; - -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Properties; -import java.util.UUID; - -import static org.junit.Assert.assertEquals; - -/** - * Tests on how the ShardConsumerThread behaves with mocked KinesisProxy behaviours. - */ -@RunWith(PowerMockRunner.class) -@PrepareForTest(ShardConsumerThread.class) -public class ShardConsumerThreadTest { - - @Test - public void testAllRecordsFetchedFromKinesisAreCorrectlyCollected() { - int totalRecordCount = 500; - - KinesisStreamShard assignedShardUnderTest = new KinesisStreamShard( - "fake-stream-name", - new Shard() - .withShardId("fake-shard-id") - .withAdjacentParentShardId(null) - .withParentShardId(null) - .withHashKeyRange(new HashKeyRange().withStartingHashKey("0").withEndingHashKey(StringUtils.repeat("FF", 16)))); - - // ------------------------------------------------------------------------------------------ - // the part below prepares the behaviour of the mocked KinesisProxy for getting the inital shard iterator, - // followed by consecutive getRecords() calls until total of 500 records fetched - // ------------------------------------------------------------------------------------------ - - KinesisProxy kinesisProxyMock = Mockito.mock(KinesisProxy.class); - Mockito.when(kinesisProxyMock.getShardIterator(Matchers.any(KinesisStreamShard.class), Matchers.anyString(), Matchers.anyString())) - .thenReturn("fake-initial-shard-itr"); - - // 1st getRecords() returns 100 records - GetRecordsResult getRecordsResultFirst = new GetRecordsResult() - .withRecords(generateFakeListOfRecordsFromToIncluding(0, 99)) - .withNextShardIterator("fake-1st-shard-itr"); - - // 2nd getRecords() returns 90 records - GetRecordsResult getRecordsResultSecond = new GetRecordsResult() - .withRecords(generateFakeListOfRecordsFromToIncluding(100, 189)) - .withNextShardIterator("fake-2nd-shard-itr"); - - // 3rd getRecords() returns 78 records - GetRecordsResult getRecordsResultThird = new GetRecordsResult() - .withRecords(generateFakeListOfRecordsFromToIncluding(190, 267)) - .withNextShardIterator("fake-3rd-shard-itr"); - - // 4th getRecords() returns 100 records - GetRecordsResult getRecordsResultFourth = new GetRecordsResult() - .withRecords(generateFakeListOfRecordsFromToIncluding(268, 367)) - .withNextShardIterator("fake-4th-shard-itr"); - - GetRecordsResult getRecordsResultFifth = new GetRecordsResult() - .withRecords(generateFakeListOfRecordsFromToIncluding(368, 459)) - .withNextShardIterator("fake-5th-shard-itr"); - - GetRecordsResult getRecordsResultFinal = new GetRecordsResult() - .withRecords(generateFakeListOfRecordsFromToIncluding(460, 499)) - .withNextShardIterator(null); - - Mockito.when(kinesisProxyMock.getRecords(Matchers.anyString(), Matchers.anyInt())) - .thenReturn(getRecordsResultFirst) - .thenReturn(getRecordsResultSecond) - .thenReturn(getRecordsResultThird) - .thenReturn(getRecordsResultFourth) - .thenReturn(getRecordsResultFifth) - .thenReturn(getRecordsResultFinal); - - // assuming that all fetched records are not aggregated, - // so we are mocking the static deaggregateRecords() to return the original list of records - PowerMockito.mockStatic(ShardConsumerThread.class); - PowerMockito.when(ShardConsumerThread.deaggregateRecords(Matchers.anyListOf(Record.class), Matchers.anyString(), Matchers.anyString())) - .thenReturn(convertRecordsToUserRecords(getRecordsResultFirst.getRecords())) - .thenReturn(convertRecordsToUserRecords(getRecordsResultSecond.getRecords())) - .thenReturn(convertRecordsToUserRecords(getRecordsResultThird.getRecords())) - .thenReturn(convertRecordsToUserRecords(getRecordsResultFourth.getRecords())) - .thenReturn(convertRecordsToUserRecords(getRecordsResultFifth.getRecords())) - .thenReturn(convertRecordsToUserRecords(getRecordsResultFinal.getRecords())); - - // ------------------------------------------------------------------------------------------ - - Properties testConsumerConfig = new Properties(); - HashMap seqNumState = new HashMap<>(); - - DummySourceContext dummySourceContext = new DummySourceContext(); - ShardConsumerThread dummyShardConsumerThread = getDummyShardConsumerThreadWithMockedKinesisProxy( - dummySourceContext, kinesisProxyMock, Mockito.mock(KinesisDataFetcher.class), - testConsumerConfig, assignedShardUnderTest, new SequenceNumber("fake-last-seq-num"), seqNumState); - - dummyShardConsumerThread.run(); - - // the final sequence number state for the assigned shard to this consumer thread - // should store SENTINEL_SHARD_ENDING_SEQUENCE_NUMBER since the final nextShardItr should be null - assertEquals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get(), seqNumState.get(assignedShardUnderTest)); - - // the number of elements collected should equal the number of records generated by mocked KinesisProxy - assertEquals(totalRecordCount, dummySourceContext.getNumOfElementsCollected()); - } - - private ShardConsumerThread getDummyShardConsumerThreadWithMockedKinesisProxy( - SourceFunction.SourceContext dummySourceContext, - KinesisProxy kinesisProxyMock, - KinesisDataFetcher owningFetcherRefMock, - Properties testConsumerConfig, - KinesisStreamShard assignedShard, - SequenceNumber lastSequenceNum, - HashMap seqNumState) { - - try { - PowerMockito.whenNew(KinesisProxy.class).withArguments(testConsumerConfig).thenReturn(kinesisProxyMock); - } catch (Exception e) { - throw new RuntimeException("Error when power mocking KinesisProxy in test", e); - } - - return new ShardConsumerThread<>(owningFetcherRefMock, testConsumerConfig, - assignedShard, lastSequenceNum, dummySourceContext, new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()), seqNumState); - } - - private List generateFakeListOfRecordsFromToIncluding(int startingSeq, int endingSeq) { - List fakeListOfRecords = new LinkedList<>(); - for (int i=startingSeq; i <= endingSeq; i++) { - fakeListOfRecords.add(new Record() - .withData(ByteBuffer.wrap(String.valueOf(i).getBytes())) - .withPartitionKey(UUID.randomUUID().toString()) // the partition key assigned doesn't matter here - .withSequenceNumber(String.valueOf(i))); // assign the order of the record within the whole sequence as the sequence num - } - return fakeListOfRecords; - } - - private List convertRecordsToUserRecords(List records) { - List converted = new ArrayList<>(records.size()); - for (Record record : records) { - converted.add(new UserRecord(record)); - } - return converted; - } - - private static class DummySourceContext implements SourceFunction.SourceContext { - private static final Object lock = new Object(); - - private static long numElementsCollected; - - public DummySourceContext() { - numElementsCollected = 0; - } - - @Override - public void collect(String element) { - numElementsCollected++; - } - - @Override - public void collectWithTimestamp(java.lang.String element, long timestamp) { - } - - @Override - public void emitWatermark(Watermark mark) { - } - - @Override - public Object getCheckpointLock() { - return lock; - } - - @Override - public void close() { - } - - public long getNumOfElementsCollected() { - return numElementsCollected; - } - } - -} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java index 955b990765627..409124c6f7e60 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java @@ -16,39 +16,22 @@ */ package org.apache.flink.streaming.connectors.kinesis.manualtests; -import com.amazonaws.regions.Region; -import com.amazonaws.regions.Regions; import com.amazonaws.services.kinesis.AmazonKinesisClient; import com.amazonaws.services.kinesis.model.DescribeStreamResult; -import org.apache.commons.lang3.RandomStringUtils; -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.RichFlatMapFunction; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.checkpoint.Checkpointed; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; -import org.apache.flink.streaming.connectors.kinesis.config.InitialPosition; import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisEventsGeneratorProducerThread; import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; -import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.flink.test.util.ForkableFlinkMiniCluster; -import org.apache.flink.test.util.SuccessException; -import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.BitSet; import java.util.Properties; import java.util.UUID; - -import static org.apache.flink.test.util.TestUtils.tryExecute; +import java.util.concurrent.atomic.AtomicReference; /** * This test first starts a data generator, producing data into kinesis. @@ -62,20 +45,26 @@ public class ManualExactlyOnceTest { private static final Logger LOG = LoggerFactory.getLogger(ManualExactlyOnceTest.class); - static final long TOTAL_EVENT_COUNT = 1000; // the producer writes one per 10 ms, so it runs for 10k ms = 10 seconds + static final int TOTAL_EVENT_COUNT = 1000; // the producer writes one per 10 ms, so it runs for 10k ms = 10 seconds public static void main(String[] args) throws Exception { final ParameterTool pt = ParameterTool.fromArgs(args); LOG.info("Starting exactly once test"); - // create a stream for the test: - Properties configProps = new Properties(); - configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accessKey")); - configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretKey")); - AmazonKinesisClient client = new AmazonKinesisClient(AWSUtil.getCredentialsProvider(configProps).getCredentials()); - client.setRegion(Region.getRegion(Regions.fromName(pt.getRequired("region")))); final String streamName = "flink-test-" + UUID.randomUUID().toString(); + final String accessKey = pt.getRequired("accessKey"); + final String secretKey = pt.getRequired("secretKey"); + final String region = pt.getRequired("region"); + + Properties configProps = new Properties(); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, accessKey); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, secretKey); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, region); + AmazonKinesisClient client = AWSUtil.createKinesisClient(configProps); + + // create a stream for the test: client.createStream(streamName, 1); + // wait until stream has been created DescribeStreamResult status = client.describeStream(streamName); LOG.info("status {}" ,status); @@ -97,92 +86,29 @@ public static void main(String[] args) throws Exception { final int flinkPort = flink.getLeaderRPCPort(); try { - final Tuple1 producerException = new Tuple1<>(); - Runnable producer = new Runnable() { - @Override - public void run() { - try { - StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); - see.setParallelism(2); - - // start data generator - DataStream simpleStringStream = see.addSource(new EventsGenerator(TOTAL_EVENT_COUNT)).setParallelism(1); - - Properties producerProps = new Properties(); - producerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accessKey")); - producerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretKey")); - producerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, pt.getRequired("region")); - - FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(new SimpleStringSchema(), - producerProps); - - kinesis.setFailOnError(true); - kinesis.setDefaultStream(streamName); - kinesis.setDefaultPartition("0"); - simpleStringStream.addSink(kinesis); - - LOG.info("Starting producing topology"); - see.execute("Producing topology"); - LOG.info("Producing topo finished"); - } catch (Exception e) { - LOG.warn("Error while running producing topology", e); - producerException.f0 = e; - } - } - }; - Thread producerThread = new Thread(producer); + final AtomicReference producerError = new AtomicReference<>(); + Thread producerThread = KinesisEventsGeneratorProducerThread.create( + TOTAL_EVENT_COUNT, 2, + accessKey, secretKey, region, streamName, + producerError, flinkPort, flinkConfig); producerThread.start(); - - final Tuple1 consumerException = new Tuple1<>(); - Runnable consumer = new Runnable() { - @Override - public void run() { - try { - StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); - see.setParallelism(2); - see.enableCheckpointing(500); - // we restart two times - see.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 500L)); - - // consuming topology - Properties consumerProps = new Properties(); - consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, pt.getRequired("accessKey")); - consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, pt.getRequired("secretKey")); - consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, pt.getRequired("region")); - // start reading from beginning - consumerProps.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, InitialPosition.TRIM_HORIZON.name()); - DataStream consuming = see.addSource(new FlinkKinesisConsumer<>(streamName, new SimpleStringSchema(), consumerProps)); - consuming.flatMap(new RichFlatMapFunction() { - int count = 0; - - @Override - public void flatMap(String value, Collector out) throws Exception { - if (count++ >= 200 && getRuntimeContext().getAttemptNumber() == 0) { - throw new RuntimeException("Artificial failure. Restart pls"); - } - out.collect(value); - } - }).flatMap(new ExactlyOnceValidatingMapper()); - // validate consumed records for correctness - LOG.info("Starting consuming topology"); - tryExecute(see, "Consuming topo"); - LOG.info("Consuming topo finished"); - } catch (Exception e) { - LOG.warn("Error while running consuming topology", e); - consumerException.f0 = e; - } - } - }; - - Thread consumerThread = new Thread(consumer); + final AtomicReference consumerError = new AtomicReference<>(); + Thread consumerThread = ExactlyOnceValidatingConsumerThread.create( + TOTAL_EVENT_COUNT, 200, 2, 500, 500, + accessKey, secretKey, region, streamName, + consumerError, flinkPort, flinkConfig); consumerThread.start(); + boolean deadlinePassed = false; long deadline = System.currentTimeMillis() + (1000 * 2 * 60); // wait at most for two minutes - while (consumerThread.isAlive() || producerThread.isAlive()) { + // wait until both producer and consumer finishes, or an unexpected error is thrown + while ((consumerThread.isAlive() || producerThread.isAlive()) && + (producerError.get() == null && consumerError.get() == null)) { Thread.sleep(1000); if (System.currentTimeMillis() >= deadline) { LOG.warn("Deadline passed"); + deadlinePassed = true; break; // enough waiting } } @@ -195,16 +121,20 @@ public void flatMap(String value, Collector out) throws Exception { consumerThread.interrupt(); } - if (producerException.f0 != null) { - throw new RuntimeException("Producer failed", producerException.f0); + if (producerError.get() != null) { + LOG.info("+++ TEST failed! +++"); + throw new RuntimeException("Producer failed", producerError.get()); } - if (consumerException.f0 != null) { - throw new RuntimeException("Consumer failed", consumerException.f0); + if (consumerError.get() != null) { + LOG.info("+++ TEST failed! +++"); + throw new RuntimeException("Consumer failed", consumerError.get()); } - - - LOG.info("+++ TEST passed! +++"); + if (!deadlinePassed) { + LOG.info("+++ TEST passed! +++"); + } else { + LOG.info("+++ TEST failed! +++"); + } } finally { client.deleteStream(streamName); @@ -214,67 +144,4 @@ public void flatMap(String value, Collector out) throws Exception { flink.stop(); } } - - // validate exactly once - private static class ExactlyOnceValidatingMapper implements FlatMapFunction, Checkpointed { - BitSet validator = new BitSet((int)TOTAL_EVENT_COUNT); - @Override - public void flatMap(String value, Collector out) throws Exception { - LOG.info("Consumed {}", value); - - int id = Integer.parseInt(value.split("-")[0]); - if(validator.get(id)) { - throw new RuntimeException("Saw id " + id +" twice!"); - } - validator.set(id); - if(id > 999) { - throw new RuntimeException("Out of bounds ID observed"); - } - - if(validator.nextClearBit(0) == 1000) { // 0 - 1000 are set - throw new SuccessException(); - } - } - - @Override - public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { - return validator; - } - - @Override - public void restoreState(BitSet state) throws Exception { - this.validator = state; - } - } - - public static class EventsGenerator implements SourceFunction { - private boolean running = true; - private final long limit; - - public EventsGenerator(long limit) { - this.limit = limit; - } - - - @Override - public void run(SourceContext ctx) throws Exception { - long seq = 0; - while(running) { - Thread.sleep(10); - String evt = (seq++) + "-" + RandomStringUtils.randomAlphabetic(12); - ctx.collect(evt); - LOG.info("Emitting event {}", evt); - if(seq >= limit) { - break; - } - } - ctx.close(); - LOG.info("Stopping events generator"); - } - - @Override - public void cancel() { - running = false; - } - } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java new file mode 100644 index 0000000000000..7bcc8064ea52e --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.manualtests; + +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread; +import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; +import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; +import org.apache.flink.test.util.ForkableFlinkMiniCluster; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This test first starts a data generator, producing data into kinesis. + * Then, it starts a consuming topology, ensuring that all records up to a certain + * point have been seen. While the data generator and consuming topology is running, + * the kinesis stream is resharded two times. + * + * Invocation: + * --region eu-central-1 --accessKey XXXXXXXXXXXX --secretKey XXXXXXXXXXXXXXXX + */ +public class ManualExactlyOnceWithStreamReshardingTest { + + private static final Logger LOG = LoggerFactory.getLogger(ManualExactlyOnceWithStreamReshardingTest.class); + + static final int TOTAL_EVENT_COUNT = 20000; // a large enough record count so we can test resharding + + public static void main(String[] args) throws Exception { + final ParameterTool pt = ParameterTool.fromArgs(args); + LOG.info("Starting exactly once with stream resharding test"); + + final String streamName = "flink-test-" + UUID.randomUUID().toString(); + final String accessKey = pt.getRequired("accessKey"); + final String secretKey = pt.getRequired("secretKey"); + final String region = pt.getRequired("region"); + + final Properties configProps = new Properties(); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, accessKey); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, secretKey); + configProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, region); + configProps.setProperty(KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS, "0"); + final AmazonKinesisClient client = AWSUtil.createKinesisClient(configProps); + + // the stream is first created with 1 shard + client.createStream(streamName, 1); + + // wait until stream has been created + DescribeStreamResult status = client.describeStream(streamName); + LOG.info("status {}", status); + while(!status.getStreamDescription().getStreamStatus().equals("ACTIVE")) { + status = client.describeStream(streamName); + LOG.info("Status of stream {}", status); + Thread.sleep(1000); + } + + final Configuration flinkConfig = new Configuration(); + flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8); + flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16); + flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); + + ForkableFlinkMiniCluster flink = new ForkableFlinkMiniCluster(flinkConfig, false); + flink.start(); + + final int flinkPort = flink.getLeaderRPCPort(); + + try { + // we have to use a manual generator here instead of the FlinkKinesisProducer + // because the FlinkKinesisProducer currently has a problem where records will be resent to a shard + // when resharding happens; this affects the consumer exactly-once validation test and will never pass + final AtomicReference producerError = new AtomicReference<>(); + Runnable manualGenerate = new Runnable() { + @Override + public void run() { + AmazonKinesisClient client = AWSUtil.createKinesisClient(configProps); + int count = 0; + final int batchSize = 30; + while (true) { + try { + Thread.sleep(10); + + Set batch = new HashSet<>(); + for (int i=count; i= TOTAL_EVENT_COUNT) { + break; + } + batch.add( + new PutRecordsRequestEntry() + .withData(ByteBuffer.wrap(((i) + "-" + RandomStringUtils.randomAlphabetic(12)).getBytes())) + .withPartitionKey(UUID.randomUUID().toString())); + } + count += batchSize; + + PutRecordsResult result = client.putRecords(new PutRecordsRequest().withStreamName(streamName).withRecords(batch)); + + // the putRecords() operation may have failing records; to keep this test simple + // instead of retrying on failed records, we simply pass on a runtime exception + // and let this test fail + if (result.getFailedRecordCount() > 0) { + producerError.set(new RuntimeException("The producer has failed records in one of the put batch attempts.")); + break; + } + + if (count >= TOTAL_EVENT_COUNT) { + break; + } + } catch (Exception e) { + producerError.set(e); + } + } + } + }; + Thread producerThread = new Thread(manualGenerate); + producerThread.start(); + + final AtomicReference consumerError = new AtomicReference<>(); + Thread consumerThread = ExactlyOnceValidatingConsumerThread.create( + TOTAL_EVENT_COUNT, 10000, 2, 500, 500, + accessKey, secretKey, region, streamName, + consumerError, flinkPort, flinkConfig); + consumerThread.start(); + + // reshard the Kinesis stream while the producer / and consumers are running + Runnable splitShard = new Runnable() { + @Override + public void run() { + try { + // first, split shard in the middle of the hash range + Thread.sleep(5000); + LOG.info("Splitting shard ..."); + client.splitShard( + streamName, + KinesisShardIdGenerator.generateFromShardOrder(0), + "170141183460469231731687303715884105727"); + + // wait until the split shard operation finishes updating ... + DescribeStreamResult status; + Random rand = new Random(); + do { + status = null; + while (status == null) { + // retry until we get status + try { + status = client.describeStream(streamName); + } catch (LimitExceededException lee) { + LOG.warn("LimitExceededException while describing stream ... retrying ..."); + Thread.sleep(rand.nextInt(1200)); + } + } + } while (!status.getStreamDescription().getStreamStatus().equals("ACTIVE")); + + // then merge again + Thread.sleep(7000); + LOG.info("Merging shards ..."); + client.mergeShards( + streamName, + KinesisShardIdGenerator.generateFromShardOrder(1), + KinesisShardIdGenerator.generateFromShardOrder(2)); + } catch (InterruptedException iex) { + // + } + } + }; + Thread splitShardThread = new Thread(splitShard); + splitShardThread.start(); + + boolean deadlinePassed = false; + long deadline = System.currentTimeMillis() + (1000 * 5 * 60); // wait at most for five minutes + // wait until both producer and consumer finishes, or an unexpected error is thrown + while ((consumerThread.isAlive() || producerThread.isAlive()) && + (producerError.get() == null && consumerError.get() == null)) { + Thread.sleep(1000); + if (System.currentTimeMillis() >= deadline) { + LOG.warn("Deadline passed"); + deadlinePassed = true; + break; // enough waiting + } + } + + if (producerThread.isAlive()) { + producerThread.interrupt(); + } + + if (consumerThread.isAlive()) { + consumerThread.interrupt(); + } + + if (producerError.get() != null) { + LOG.info("+++ TEST failed! +++"); + throw new RuntimeException("Producer failed", producerError.get()); + + } + + if (consumerError.get() != null) { + LOG.info("+++ TEST failed! +++"); + throw new RuntimeException("Consumer failed", consumerError.get()); + } + + if (!deadlinePassed) { + LOG.info("+++ TEST passed! +++"); + } else { + LOG.info("+++ TEST failed! +++"); + } + + } finally { + client.deleteStream(streamName); + client.shutdown(); + + // stopping flink + flink.stop(); + } + } + +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java new file mode 100644 index 0000000000000..911710f8f3b29 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ExactlyOnceValidatingConsumerThread.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.testutils; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.checkpoint.Checkpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.config.InitialPosition; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.BitSet; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.test.util.TestUtils.tryExecute; + +/** + * A thread that runs a topology with the FlinkKinesisConsumer as source, followed by two flat map + * functions, one that performs artificial failures and another that validates exactly-once guarantee + */ +public class ExactlyOnceValidatingConsumerThread { + + private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingConsumerThread.class); + + public static Thread create(final int totalEventCount, + final int failAtRecordCount, + final int parallelism, + final int checkpointInterval, + final long restartDelay, + final String awsAccessKey, + final String awsSecretKey, + final String awsRegion, + final String kinesisStreamName, + final AtomicReference errorHandler, + final int flinkPort, + final Configuration flinkConfig) { + Runnable exactlyOnceValidationConsumer = new Runnable() { + @Override + public void run() { + try { + StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); + see.setParallelism(parallelism); + see.enableCheckpointing(checkpointInterval); + // we restart two times + see.setRestartStrategy(RestartStrategies.fixedDelayRestart(2, restartDelay)); + + // consuming topology + Properties consumerProps = new Properties(); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, awsAccessKey); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, awsSecretKey); + consumerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, awsRegion); + // start reading from beginning + consumerProps.setProperty(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, InitialPosition.TRIM_HORIZON.name()); + DataStream consuming = see.addSource(new FlinkKinesisConsumer<>(kinesisStreamName, new SimpleStringSchema(), consumerProps)); + consuming + .flatMap(new ArtificialFailOnceFlatMapper(failAtRecordCount)) + // validate consumed records for correctness (use only 1 instance to validate all consumed records) + .flatMap(new ExactlyOnceValidatingMapper(totalEventCount)).setParallelism(1); + + LOG.info("Starting consuming topology"); + tryExecute(see, "Consuming topo"); + LOG.info("Consuming topo finished"); + } catch (Exception e) { + LOG.warn("Error while running consuming topology", e); + errorHandler.set(e); + } + } + }; + + return new Thread(exactlyOnceValidationConsumer); + } + + private static class ExactlyOnceValidatingMapper implements FlatMapFunction, Checkpointed { + + private static final Logger LOG = LoggerFactory.getLogger(ExactlyOnceValidatingMapper.class); + + private final int totalEventCount; + private BitSet validator; + + public ExactlyOnceValidatingMapper(int totalEventCount) { + this.totalEventCount = totalEventCount; + this.validator = new BitSet(totalEventCount); + } + + @Override + public void flatMap(String value, Collector out) throws Exception { + LOG.info("Consumed {}", value); + + int id = Integer.parseInt(value.split("-")[0]); + if(validator.get(id)) { + throw new RuntimeException("Saw id " + id +" twice!"); + } + validator.set(id); + if(id > totalEventCount-1) { + throw new RuntimeException("Out of bounds ID observed"); + } + + if(validator.nextClearBit(0) == totalEventCount) { + throw new SuccessException(); + } + } + + @Override + public BitSet snapshotState(long checkpointId, long checkpointTimestamp) throws Exception { + return validator; + } + + @Override + public void restoreState(BitSet state) throws Exception { + this.validator = state; + } + } + + private static class ArtificialFailOnceFlatMapper extends RichFlatMapFunction { + int count = 0; + + private final int failAtRecordCount; + + public ArtificialFailOnceFlatMapper(int failAtRecordCount) { + this.failAtRecordCount = failAtRecordCount; + } + + @Override + public void flatMap(String value, Collector out) throws Exception { + if (count++ >= failAtRecordCount && getRuntimeContext().getAttemptNumber() == 0) { + throw new RuntimeException("Artificial failure. Restart please."); + } + out.collect(value); + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java new file mode 100644 index 0000000000000..82509dab812bc --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/FakeKinesisBehavioursFactory.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.testutils; + +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.Shard; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +/** + * Factory for different kinds of fake Kinesis behaviours using the {@link KinesisProxyInterface} interface. + */ +public class FakeKinesisBehavioursFactory { + + // ------------------------------------------------------------------------ + // Behaviours related to shard listing and resharding, used in ShardDiscovererTest + // ------------------------------------------------------------------------ + + public static KinesisProxyInterface noShardsFoundForRequestedStreamsBehaviour() { + + return new KinesisProxyInterface() { + @Override + public GetShardListResult getShardList(Map streamNamesWithLastSeenShardIds) { + return new GetShardListResult(); // not setting any retrieved shards for result + } + + @Override + public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) { + return null; + } + + @Override + public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) { + return null; + } + }; + + } + + public static KinesisProxyInterface nonReshardedStreamsBehaviour(Map streamsToShardCount) { + return new NonReshardedStreamsKinesis(streamsToShardCount); + + } + + // ------------------------------------------------------------------------ + // Behaviours related to fetching records, used mainly in ShardConsumerTest + // ------------------------------------------------------------------------ + + public static KinesisProxyInterface totalNumOfRecordsAfterNumOfGetRecordsCalls(final int numOfRecords, final int numOfGetRecordsCalls) { + return new SingleShardEmittingFixNumOfRecordsKinesis(numOfRecords, numOfGetRecordsCalls); + } + + private static class SingleShardEmittingFixNumOfRecordsKinesis implements KinesisProxyInterface { + + private final int totalNumOfGetRecordsCalls; + + private final int totalNumOfRecords; + + private final Map> shardItrToRecordBatch; + + public SingleShardEmittingFixNumOfRecordsKinesis(final int numOfRecords, final int numOfGetRecordsCalls) { + this.totalNumOfRecords = numOfRecords; + this.totalNumOfGetRecordsCalls = numOfGetRecordsCalls; + + // initialize the record batches that we will be fetched + this.shardItrToRecordBatch = new HashMap<>(); + + int numOfAlreadyPartitionedRecords = 0; + int numOfRecordsPerBatch = numOfRecords/numOfGetRecordsCalls + 1; + for (int batch=0; batch streamNamesWithLastSeenShardIds) { + return null; + } + + public static List createRecordBatchWithRange(int min, int max) { + List batch = new LinkedList<>(); + for (int i = min; i < max; i++) { + batch.add( + new Record() + .withData(ByteBuffer.wrap(String.valueOf(i).getBytes())) + .withPartitionKey(UUID.randomUUID().toString()) + .withSequenceNumber(String.valueOf(i))); + } + return batch; + } + + } + + private static class NonReshardedStreamsKinesis implements KinesisProxyInterface { + + private Map> streamsWithListOfShards = new HashMap<>(); + + public NonReshardedStreamsKinesis(Map streamsToShardCount) { + for (Map.Entry streamToShardCount : streamsToShardCount.entrySet()) { + String streamName = streamToShardCount.getKey(); + int shardCount = streamToShardCount.getValue(); + + if (shardCount == 0) { + // don't do anything + } else { + List shardsOfStream = new ArrayList<>(shardCount); + for (int i=0; i < shardCount; i++) { + shardsOfStream.add( + new KinesisStreamShard( + streamName, + new Shard().withShardId(KinesisShardIdGenerator.generateFromShardOrder(i)))); + } + streamsWithListOfShards.put(streamName, shardsOfStream); + } + } + } + + @Override + public GetShardListResult getShardList(Map streamNamesWithLastSeenShardIds) { + GetShardListResult result = new GetShardListResult(); + for (Map.Entry> streamsWithShards : streamsWithListOfShards.entrySet()) { + String streamName = streamsWithShards.getKey(); + for (KinesisStreamShard shard : streamsWithShards.getValue()) { + if (streamNamesWithLastSeenShardIds.get(streamName) == null) { + result.addRetrievedShardToStream(streamName, shard); + } else { + if (KinesisStreamShard.compareShardIds( + shard.getShard().getShardId(), streamNamesWithLastSeenShardIds.get(streamName)) > 0) { + result.addRetrievedShardToStream(streamName, shard); + } + } + } + } + return result; + } + + @Override + public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) { + return null; + } + + @Override + public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) { + return null; + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java new file mode 100644 index 0000000000000..696d9ca2b68d8 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisEventsGeneratorProducerThread.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.testutils; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; + +/** + * A thread that runs a topology with a manual data generator as source, and the FlinkKinesisProducer as sink. + */ +public class KinesisEventsGeneratorProducerThread { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisEventsGeneratorProducerThread.class); + + public static Thread create(final int totalEventCount, + final int parallelism, + final String awsAccessKey, + final String awsSecretKey, + final String awsRegion, + final String kinesisStreamName, + final AtomicReference errorHandler, + final int flinkPort, + final Configuration flinkConfig) { + Runnable kinesisEventsGeneratorProducer = new Runnable() { + @Override + public void run() { + try { + StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); + see.setParallelism(parallelism); + + // start data generator + DataStream simpleStringStream = see.addSource(new KinesisEventsGeneratorProducerThread.EventsGenerator(totalEventCount)).setParallelism(1); + + Properties producerProps = new Properties(); + producerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_ACCESSKEYID, awsAccessKey); + producerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_BASIC_SECRETKEY, awsSecretKey); + producerProps.setProperty(KinesisConfigConstants.CONFIG_AWS_REGION, awsRegion); + + FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(new SimpleStringSchema(), + producerProps); + + kinesis.setFailOnError(true); + kinesis.setDefaultStream(kinesisStreamName); + kinesis.setDefaultPartition("0"); + simpleStringStream.addSink(kinesis); + + LOG.info("Starting producing topology"); + see.execute("Producing topology"); + LOG.info("Producing topo finished"); + } catch (Exception e) { + LOG.warn("Error while running producing topology", e); + errorHandler.set(e); + } + } + }; + + return new Thread(kinesisEventsGeneratorProducer); + } + + private static class EventsGenerator implements SourceFunction { + + private static final Logger LOG = LoggerFactory.getLogger(EventsGenerator.class); + + private boolean running = true; + private final long limit; + + public EventsGenerator(long limit) { + this.limit = limit; + } + + @Override + public void run(SourceContext ctx) throws Exception { + long seq = 0; + while(running) { + Thread.sleep(10); + String evt = (seq++) + "-" + RandomStringUtils.randomAlphabetic(12); + ctx.collect(evt); + LOG.info("Emitting event {}", evt); + if(seq >= limit) { + break; + } + } + ctx.close(); + LOG.info("Stopping events generator"); + } + + @Override + public void cancel() { + running = false; + } + } +} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java index 1160a6de66b67..c8dd347fe98ec 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesisShardIdGenerator.java @@ -18,7 +18,8 @@ package org.apache.flink.streaming.connectors.kinesis.testutils; public class KinesisShardIdGenerator { + // Kinesis shards ids are in the form of: ^shardId-\d{12} public static String generateFromShardOrder(int order) { - return String.format("shard-%05d", order); + return String.format("shardId-%012d", order); } } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java deleted file mode 100644 index c6e8a412d4ec5..0000000000000 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/ReferenceKinesisShardTopologies.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.testutils; - -import com.amazonaws.services.kinesis.model.SequenceNumberRange; -import com.amazonaws.services.kinesis.model.Shard; -import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; -import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator; - -import java.util.ArrayList; -import java.util.List; - -/** - * Util class to help generate example shard topologies for testing. - */ -public class ReferenceKinesisShardTopologies { - - private static final String DEFAULT_STREAM = "flink-kinesis-test"; - - /** - * A basic topology with 4 shards, where each shard is still open, - * and have no parent-child relationships due to shard split or merge. - * - * Topology layout: - * - * +- shard 0 (seq: 0 ~ open) - * | - * +- shard 1 (seq: 250 ~ open) - * | - * +- shard 2 (seq: 500 ~ open) - * | - * +- shard 3 (seq: 750 ~ open) - * - */ - public static List flatTopologyWithFourOpenShards() { - int shardCount = 4; - List topology = new ArrayList<>(shardCount); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("0")))); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(1)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("250")))); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(2)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("500")))); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(3)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("750")))); - return topology; - } - - /** - * A basic topology with 4 shards, where each shard is still open, - * and have no parent-child relationships due to shard split or merge. - * - * Topology layout: - * - * +- shard 0 (seq: 0 ~ 120) --+ - * | +- (merge) -- shard 3 (750 ~ open) - * +- shard 1 (seq: 250 ~ 289) --+ - * | - * +- shard 2 (seq: 500 ~ open) - * - */ - public static List topologyWithThreeInitialShardsAndFirstTwoMerged() { - int shardCount = 4; - - List topology = new ArrayList<>(shardCount); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(0)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("0").withEndingSequenceNumber("120")))); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(1)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("250").withEndingSequenceNumber("289")))); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(2)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("500")))); - topology.add(new KinesisStreamShard( - DEFAULT_STREAM, - new Shard() - .withShardId(KinesisShardIdGenerator.generateFromShardOrder(3)) - .withSequenceNumberRange(new SequenceNumberRange().withStartingSequenceNumber("750")))); - - return topology; - } - -} diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java index f63ec987dd084..80ad06c928743 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableFlinkKinesisConsumer.java @@ -18,41 +18,43 @@ package org.apache.flink.streaming.connectors.kinesis.testutils; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.util.Properties; -/** - * A testable FlinkKinesisConsumer that overrides getRuntimeContext to return a dummy StreamRuntimeContext. - */ -public class TestableFlinkKinesisConsumer extends FlinkKinesisConsumer { - - private final int fakeNumFlinkConsumerTasks; - private final int fakeThisConsumerTaskIndex; - private final String fakeThisConsumerTaskName; - - - public TestableFlinkKinesisConsumer(String fakeStreamName, - int fakeNumFlinkConsumerTasks, - int fakeThisConsumerTaskIndex, - String fakeThisConsumerTaskName, - Properties configProps) { - super(fakeStreamName, new SimpleStringSchema(), configProps); - this.fakeNumFlinkConsumerTasks = fakeNumFlinkConsumerTasks; - this.fakeThisConsumerTaskIndex = fakeThisConsumerTaskIndex; - this.fakeThisConsumerTaskName = fakeThisConsumerTaskName; +public class TestableFlinkKinesisConsumer extends FlinkKinesisConsumer { + + private final RuntimeContext mockedRuntimeCtx; + + public TestableFlinkKinesisConsumer(String fakeStream, + Properties fakeConfiguration, + final int totalNumOfConsumerSubtasks, + final int indexOfThisConsumerSubtask) { + super(fakeStream, new SimpleStringSchema(), fakeConfiguration); + + this.mockedRuntimeCtx = Mockito.mock(RuntimeContext.class); + + Mockito.when(mockedRuntimeCtx.getNumberOfParallelSubtasks()).thenAnswer(new Answer() { + @Override + public Integer answer(InvocationOnMock invocationOnMock) throws Throwable { + return totalNumOfConsumerSubtasks; + } + }); + + Mockito.when(mockedRuntimeCtx.getIndexOfThisSubtask()).thenAnswer(new Answer() { + @Override + public Integer answer(InvocationOnMock invocationOnMock) throws Throwable { + return indexOfThisConsumerSubtask; + } + }); } @Override public RuntimeContext getRuntimeContext() { - StreamingRuntimeContext runtimeContextMock = Mockito.mock(StreamingRuntimeContext.class); - Mockito.when(runtimeContextMock.getNumberOfParallelSubtasks()).thenReturn(fakeNumFlinkConsumerTasks); - Mockito.when(runtimeContextMock.getIndexOfThisSubtask()).thenReturn(fakeThisConsumerTaskIndex); - Mockito.when(runtimeContextMock.getTaskName()).thenReturn(fakeThisConsumerTaskName); - return runtimeContextMock; + return this.mockedRuntimeCtx; } - } diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java new file mode 100644 index 0000000000000..765ff7e670497 --- /dev/null +++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/TestableKinesisDataFetcher.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License 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 org.apache.flink.streaming.connectors.kinesis.testutils; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; +import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; + +public class TestableKinesisDataFetcher extends KinesisDataFetcher { + + private static final Object fakeCheckpointLock = new Object(); + + private long numElementsCollected; + + public TestableKinesisDataFetcher(List fakeStreams, + Properties fakeConfiguration, + int fakeTotalCountOfSubtasks, + int fakeTndexOfThisSubtask, + AtomicReference thrownErrorUnderTest, + LinkedList subscribedShardsStateUnderTest, + HashMap subscribedStreamsToLastDiscoveredShardIdsStateUnderTest, + KinesisProxyInterface fakeKinesis) { + super(fakeStreams, + getMockedSourceContext(), + fakeCheckpointLock, + getMockedRuntimeContext(fakeTotalCountOfSubtasks, fakeTndexOfThisSubtask), + fakeConfiguration, + new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()), + thrownErrorUnderTest, + subscribedShardsStateUnderTest, + subscribedStreamsToLastDiscoveredShardIdsStateUnderTest, + fakeKinesis); + + this.numElementsCollected = 0; + } + + public long getNumOfElementsCollected() { + return numElementsCollected; + } + + @Override + protected KinesisDeserializationSchema getClonedDeserializationSchema() { + return new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()); + } + + @Override + protected void emitRecordAndUpdateState(String record, int shardStateIndex, SequenceNumber lastSequenceNumber) { + synchronized (fakeCheckpointLock) { + this.numElementsCollected++; + updateState(shardStateIndex, lastSequenceNumber); + } + } + + @SuppressWarnings("unchecked") + private static SourceFunction.SourceContext getMockedSourceContext() { + return Mockito.mock(SourceFunction.SourceContext.class); + } + + private static RuntimeContext getMockedRuntimeContext(final int fakeTotalCountOfSubtasks, final int fakeTndexOfThisSubtask) { + RuntimeContext mockedRuntimeContext = Mockito.mock(RuntimeContext.class); + + Mockito.when(mockedRuntimeContext.getNumberOfParallelSubtasks()).thenAnswer(new Answer() { + @Override + public Integer answer(InvocationOnMock invocationOnMock) throws Throwable { + return fakeTotalCountOfSubtasks; + } + }); + + Mockito.when(mockedRuntimeContext.getIndexOfThisSubtask()).thenAnswer(new Answer() { + @Override + public Integer answer(InvocationOnMock invocationOnMock) throws Throwable { + return fakeTndexOfThisSubtask; + } + }); + + Mockito.when(mockedRuntimeContext.getTaskName()).thenAnswer(new Answer() { + @Override + public String answer(InvocationOnMock invocationOnMock) throws Throwable { + return "Fake Task"; + } + }); + + Mockito.when(mockedRuntimeContext.getTaskNameWithSubtasks()).thenAnswer(new Answer() { + @Override + public String answer(InvocationOnMock invocationOnMock) throws Throwable { + return "Fake Task (" + fakeTndexOfThisSubtask + "/" + fakeTotalCountOfSubtasks + ")"; + } + }); + + return mockedRuntimeContext; + } +} From f29193e1395b7afd2411193980fdbba63ed645ca Mon Sep 17 00:00:00 2001 From: Gordon Tai Date: Fri, 1 Jul 2016 23:00:36 +0800 Subject: [PATCH 2/2] [FLINK-3231] Address last PR comments --- docs/apis/streaming/connectors/kinesis.md | 80 +++++++++++++++++-- .../kinesis/internals/KinesisDataFetcher.java | 23 ++++-- 2 files changed, 92 insertions(+), 11 deletions(-) diff --git a/docs/apis/streaming/connectors/kinesis.md b/docs/apis/streaming/connectors/kinesis.md index a539956867911..1fc9961759794 100644 --- a/docs/apis/streaming/connectors/kinesis.md +++ b/docs/apis/streaming/connectors/kinesis.md @@ -114,13 +114,83 @@ the AWS access key ID and secret key are directly supplied in the configuration from the newest position in the Kinesis stream (the other option will be setting `KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE` to `TRIM_HORIZON`, which lets the consumer start reading the Kinesis stream from the earliest record possible). -Note that the consumer attempts to discover new Kinesis shards due to resharding at a fixed default interval of -10 seconds. In other words, new shards may take up to 10 seconds to be discovered. This setting can be overridden -to other values by setting a different value for `KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS` in the -supplied consumer configuration. - Other optional configuration keys can be found in `KinesisConfigConstants`. +#### Fault Tolerance for Exactly-Once User-Defined State Update Semantics + +With Flink's checkpointing enabled, the Flink Kinesis Consumer will consume records from shards in Kinesis streams and +periodically checkpoint each shard's progress. In case of a job failure, Flink will restore the streaming program to the +state of the latest complete checkpoint and re-consume the records from Kinesis shards, starting from the progress that +was stored in the checkpoint. + +The interval of drawing checkpoints therefore defines how much the program may have to go back at most, in case of a failure. + +To use fault tolerant Kinesis Consumers, checkpointing of the topology needs to be enabled at the execution environment: + +
+
+{% highlight java %} +final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +env.enableCheckpointing(5000); // checkpoint every 5000 msecs +{% endhighlight %} +
+
+{% highlight scala %} +val env = StreamExecutionEnvironment.getExecutionEnvironment() +env.enableCheckpointing(5000) // checkpoint every 5000 msecs +{% endhighlight %} +
+
+ +Also note that Flink can only restart the topology if enough processing slots are available to restart the topology. +Therefore, if the topology fails due to loss of a TaskManager, there must still be enough slots available afterwards. +Flink on YARN supports automatic restart of lost YARN containers. + +#### Threading Model + +The Flink Kinesis Consumer uses multiple threads for shard discovery and data consumption. + +For shard discovery, each parallel consumer subtask will have a single thread that constantly queries Kinesis for shard +information even if the subtask initially did not have shards to read from when the consumer was started. In other words, if +the consumer is run with a parallelism of 10, there will be a total of 10 threads constantly querying Kinesis regardless +of the total amount of shards in the subscribed streams. + +For data consumption, a single thread will be created to consume each discovered shard. Threads will terminate when the +shard it is responsible of consuming is closed as a result of stream resharding. In other words, there will always be +one thread per open shard. + +#### Internally Used Kinesis APIs + +The Flink Kinesis Consumer uses the [AWS Java SDK](http://aws.amazon.com/sdk-for-java/) internally to call Kinesis APIs +for shard discovery and data consumption. Due to Amazon's [service limits for Kinesis Streams](http://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html) +on the APIs, the consumer will be competing with other non-Flink consuming applications that the user may be running. +Below is a list of APIs called by the consumer with description of how the consumer uses the API, as well as information +on how to deal with any errors or warnings that the Flink Kinesis Consumer may have due to these service limits. + +- *[DescribeStream](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_DescribeStream.html)*: this is constantly called +by a single thread in each parallel consumer subtask to discover any new shards as a result of stream resharding. By default, +the consumer performs the shard discovery at an interval of 10 seconds, and will retry indefinitely until it gets a result +from Kinesis. If this interferes with other non-Flink consuming applications, users can slow down the consumer of +calling this API by setting a value for `KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS` in the supplied +configuration properties. This sets the discovery interval to a different value. Note that this setting directly impacts +the maximum delay of discovering a new shard and starting to consume it, as shards will not be discovered during the interval. + +- *[GetShardIterator](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetShardIterator.html)*: this is called +only once when per shard consuming threads are started, and will retry if Kinesis complains that the transaction limit for the +API has exceeded, up to a default of 3 attempts. Note that since the rate limit for this API is per shard (not per stream), +the consumer itself should not exceed the limit. Usually, if this happens, users can either try to slow down any other +non-Flink consuming applications of calling this API, or modify the retry behaviour of this API call in the consumer by +setting keys prefixed by `KinesisConfigConstants.CONFIG_SHARD_GETITERATOR_*` in the supplied configuration properties. + +- *[GetRecords](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html)*: this is constantly called +by per shard consuming threads to fetch records from Kinesis. When a shard has multiple concurrent consumers (when there +are any other non-Flink consuming applications running), the per shard rate limit may be exceeded. By default, on each call +of this API, the consumer will retry if Kinesis complains that the data size / transaction limit for the API has exceeded, +up to a default of 3 attempts. Users can either try to slow down other non-Flink consuming applications, or adjust the maximum +amount of records to fetch per call by setting the `KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_MAX` key in the supplied +configuration properties. The retry behaviour of the consumer when calling this API can also be modified by using the +other keys prefixed by `KinesisConfigConstants.CONFIG_SHARD_GETRECORDS_*`. + ### Kinesis Producer The `FlinkKinesisProducer` is used for putting data from a Flink stream into a Kinesis stream. Note that the producer is not participating in diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 269df556ba0fa..ab43f63a6fad4 100644 --- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -228,6 +228,14 @@ public void runFetcher() throws Exception { // 1. query for any new shards that may have been created while the Kinesis consumer was not running, // and register them to the subscribedShardState list. + if (LOG.isDebugEnabled()) { + String logFormat = (isRestoredFromFailure) + ? "Subtask {} is trying to discover initial shards ..." + : "Subtask {} is trying to discover any new shards that were created while the consumer wasn't" + + "running due to failure ..."; + + LOG.debug(logFormat, indexOfThisConsumerSubtask); + } List newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe(); for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) { // the starting state for new shards created while the consumer wasn't running depends on whether or not @@ -244,7 +252,7 @@ public void runFetcher() throws Exception { : "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" + "running due to failure, starting state set as sequence number {}"; - LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get()); + LOG.info(logFormat, indexOfThisConsumerSubtask, shard.toString(), startingStateForNewShard.get()); } registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get())); } @@ -277,8 +285,8 @@ public void runFetcher() throws Exception { KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex); if (LOG.isInfoEnabled()) { - LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" + - runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(), + LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}", + indexOfThisConsumerSubtask, seededShardState.getKinesisStreamShard().toString(), seededShardState.getLastProcessedSequenceNum(), seededStateIndex); } @@ -301,6 +309,10 @@ public void runFetcher() throws Exception { Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS))); while (running) { + if (LOG.isDebugEnabled()) { + LOG.debug("Subtask {} is trying to discover new shards that were created due to resharding ...", + indexOfThisConsumerSubtask); + } List newShardsDueToResharding = discoverNewShardsToSubscribe(); for (KinesisStreamShard shard : newShardsDueToResharding) { @@ -313,7 +325,7 @@ public void runFetcher() throws Exception { if (LOG.isInfoEnabled()) { LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " + "the shard from sequence number {} with ShardConsumer {}", - runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(), + indexOfThisConsumerSubtask, newShardState.getKinesisStreamShard().toString(), newShardState.getLastProcessedSequenceNum(), newStateIndex); } @@ -377,8 +389,7 @@ public void shutdownFetcher() { mainThread.interrupt(); // the main thread may be sleeping for the discovery interval if (LOG.isInfoEnabled()) { - LOG.info("Shutting down the shard consumer threads of subtask {} ...", - runtimeContext.getIndexOfThisSubtask()); + LOG.info("Shutting down the shard consumer threads of subtask {} ...", indexOfThisConsumerSubtask); } shardConsumersExecutor.shutdownNow(); }