You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/02/02 08:08:51 UTC
[jira] [Commented] (FLINK-4574) Strengthen fetch interval
implementation in Kinesis consumer
[ https://issues.apache.org/jira/browse/FLINK-4574?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15849621#comment-15849621 ]
ASF GitHub Bot commented on FLINK-4574:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/2925#discussion_r99067451
--- Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java ---
@@ -154,42 +166,115 @@ public void run() {
}
}
- // set the nextShardItr so we can continue iterating in the next while loop
- nextShardItr = getRecordsResult.getNextShardIterator();
+ // set the startShardItr so we can continue iterating in the next while loop
+ startShardItr = getRecordsResult.getNextShardIterator();
} else {
// the last record was non-aggregated, so we can simply start from the next record
- nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
+ startShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(), lastSequenceNum.getSequenceNumber());
}
}
- while(isRunning()) {
- if (nextShardItr == null) {
- 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 {
- if (fetchIntervalMillis != 0) {
- Thread.sleep(fetchIntervalMillis);
- }
+ ArrayBlockingQueue<UserRecord> queue = new ArrayBlockingQueue<>(maxNumberOfRecordsPerFetch);
+ ShardConsumerFetcher shardConsumerFetcher;
- GetRecordsResult getRecordsResult = getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
+ if (fetchIntervalMillis > 0L) {
+ shardConsumerFetcher = new ShardConsumerFetcher(this, startShardItr, queue, false);
+ timer.scheduleAtFixedRate(shardConsumerFetcher, 0L, fetchIntervalMillis);
+ } else {
+ // if fetchIntervalMillis is 0, make the task run forever and schedule it once only.
+ shardConsumerFetcher = new ShardConsumerFetcher(this, startShardItr, queue, true);
+ timer.schedule(shardConsumerFetcher, 0L);
+ }
- // each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
- List<UserRecord> fetchedRecords = deaggregateRecords(
- getRecordsResult.getRecords(),
- subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
- subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
+ while(isRunning()) {
+ UserRecord record = queue.poll();
+ if (record != null) {
+ deserializeRecordForCollectionAndUpdateState(record);
+ } else {
+ if (shardConsumerFetcher.nextShardItr == null) {
+ fetcherRef.updateState(subscribedShardStateIndex, SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get());
- for (UserRecord record : fetchedRecords) {
- deserializeRecordForCollectionAndUpdateState(record);
+ // we can close this consumer thread once we've reached the end of the subscribed shard
+ break;
}
+ }
- nextShardItr = getRecordsResult.getNextShardIterator();
+ Throwable throwable = this.error.get();
+ if (throwable != null) {
+ throw throwable;
}
}
} catch (Throwable t) {
fetcherRef.stopWithError(t);
+ } finally {
+ timer.cancel();
+ }
+ }
+
+ private class ShardConsumerFetcher extends TimerTask {
+ private String nextShardItr;
+
+ private final ShardConsumer<T> shardConsumerRef;
+
+ private final ArrayBlockingQueue<UserRecord> userRecordQueue;
+
+ /** The latest finish time for fetching data from Kinesis used to recognize if the following task has been delayed.*/
+ private Long lastFinishTime = -1L;
+
+ private boolean runForever;
+
+ ShardConsumerFetcher(ShardConsumer<T> shardConsumerRef,
+ String nextShardItr,
+ ArrayBlockingQueue<UserRecord> userRecordQueue,
+ boolean runForever) {
+ this.shardConsumerRef = shardConsumerRef;
+ this.nextShardItr = nextShardItr;
+ this.userRecordQueue = userRecordQueue;
+ this.runForever = runForever;
+ }
+
+ @Override
+ public void run() {
+ try {
+ do {
+ if (nextShardItr != null) {
+ // ignore to log this warning if runForever is true, since fetchIntervalMillis is 0
+ if (!runForever && this.scheduledExecutionTime() < lastFinishTime) {
+ // If expected scheduled execution time is earlier than lastFinishTime,
+ // it seems that the fetchIntervalMillis might be short to finish the previous task.
+ LOG.warn("The value given for ShardConsumer is too short to finish getRecords on time. Please increase the value set in config \"{}\"", ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS);
+ } else {
+ GetRecordsResult getRecordsResult = shardConsumerRef.getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
--- End diff --
Can we declare `GetRecordsResult getRecordsResult` outside of the do-while scope and reuse the variable?
> Strengthen fetch interval implementation in Kinesis consumer
> ------------------------------------------------------------
>
> Key: FLINK-4574
> URL: https://issues.apache.org/jira/browse/FLINK-4574
> Project: Flink
> Issue Type: Improvement
> Components: Kinesis Connector
> Affects Versions: 1.1.0
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Wei-Che Wei
>
> As pointed out by [~rmetzger], right now the fetch interval implementation in the {{ShardConsumer}} class of the Kinesis consumer can lead to much longer interval times than specified by the user, ex. say the specified fetch interval is {{f}}, it takes {{x}} to complete a {{getRecords()}} call, and {{y}} to complete processing the fetched records for emitting, than the actual interval between each fetch is actually {{f+x+y}}.
> The main problem with this is that we can never guarantee how much time has past since the last {{getRecords}} call, thus can not guarantee that returned shard iterators will not have expired the next time we use them, even if we limit the user-given value for {{f}} to not be longer than the iterator expire time.
> I propose to improve this by, per {{ShardConsumer}}, use a {{ScheduledExecutorService}} / {{Timer}} to do the fixed-interval fetching, and a separate blocking queue that collects the fetched records for emitting.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)