Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-4574

Strengthen fetch interval implementation in Kinesis consumer

    Details

    • Type: Improvement
    • Status: In Progress
    • Priority: Major
    • Resolution: Unresolved
    • Affects Version/s: 1.1.0
    • Fix Version/s: None
    • Component/s: Kinesis Connector
    • Labels:
      None

      Description

      As pointed out by Robert Metzger, 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.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user tony810430 opened a pull request:

          https://github.com/apache/flink/pull/2925

          FLINK-4574 [kinesis]Strengthen fetch interval implementation in Kinesis consumer

          I used Timer to implement it.

          If "flink.shard.getrecords.intervalmillis" is set by default value, which is 0, the timer will schedule ShardConsumerFetcher once and run it forever.
          If "flink.shard.getrecords.intervalmillis" is greater than 0, the timer will schedule ShardConsumerFetcher at a fixed ratio by using timer.scheduleAtFixedRate, which makes sure two consecutive function call would be a fixed interval.
          But if the getRecords took too much time and couldn't be finished on time, ShardConsumerFetcher would log the warning and drop the next delayed task.

          Ideally :

          ---p1--- ---p2--- ---p3---
          =====> ====> ====>

          task1 task2 task3

          task2 is delayed by task1: task2 will be dropped

          ---p1--- ---p2--- ---p3---
          ============> ====>

          task1 task3

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/tony810430/flink FLINK-4574

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/2925.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #2925


          commit 44ef9b4df7e805a8f20b31ba55f511264820b7c1
          Author: 魏偉哲 <tonywei@tonyweis-macbook-pro.local>
          Date: 2016-12-02T09:15:28Z

          FLINK-4574 Strengthen fetch interval implementation in Kinesis consumer


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user tony810430 opened a pull request: https://github.com/apache/flink/pull/2925 FLINK-4574 [kinesis] Strengthen fetch interval implementation in Kinesis consumer I used Timer to implement it. If "flink.shard.getrecords.intervalmillis" is set by default value, which is 0, the timer will schedule ShardConsumerFetcher once and run it forever. If "flink.shard.getrecords.intervalmillis" is greater than 0, the timer will schedule ShardConsumerFetcher at a fixed ratio by using timer.scheduleAtFixedRate, which makes sure two consecutive function call would be a fixed interval. But if the getRecords took too much time and couldn't be finished on time, ShardConsumerFetcher would log the warning and drop the next delayed task. Ideally : --- p1 --- --- p2 --- --- p3 --- =====> ====> ====> task1 task2 task3 task2 is delayed by task1: task2 will be dropped --- p1 --- --- p2 --- --- p3 --- ============> ====> task1 task3 You can merge this pull request into a Git repository by running: $ git pull https://github.com/tony810430/flink FLINK-4574 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2925.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #2925 commit 44ef9b4df7e805a8f20b31ba55f511264820b7c1 Author: 魏偉哲 <tonywei@tonyweis-macbook-pro.local> Date: 2016-12-02T09:15:28Z FLINK-4574 Strengthen fetch interval implementation in Kinesis consumer
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/2925

          @tzulitai I think it would be great to get your input on this pull request, when you find the time.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/2925 @tzulitai I think it would be great to get your input on this pull request, when you find the time.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2925

          I'm definitely planning to look at this over the next few days Currently quite overwhelmed right now.

          Thanks for all your recent work on the Kinesis connector @tony810430, and very sorry for the late reviews. Please bear with me for a little while, I'll get back to the PRs soon

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2925 I'm definitely planning to look at this over the next few days Currently quite overwhelmed right now. Thanks for all your recent work on the Kinesis connector @tony810430, and very sorry for the late reviews. Please bear with me for a little while, I'll get back to the PRs soon
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2925

          Thanks for working on this @tony810430!

          I'd like to discuss a bit on the design before a more detailed review for this PR. If I'm correct, with this PR we will be creating a separate shard consuming thread with the Timer even if the fetch interval is 0 (which is our default). In this case, especially that it's our default, this extra thread usage seems sub-optimal to me, and I'm wondering if there's a better design for this. What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2925 Thanks for working on this @tony810430! I'd like to discuss a bit on the design before a more detailed review for this PR. If I'm correct, with this PR we will be creating a separate shard consuming thread with the Timer even if the fetch interval is 0 (which is our default). In this case, especially that it's our default, this extra thread usage seems sub-optimal to me, and I'm wondering if there's a better design for this. What do you think?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tony810430 commented on the issue:

          https://github.com/apache/flink/pull/2925

          I think it would not be sub-optimal, because it still need to separate consuming thread and ingesting thread, which emits records to next operator. The reason for this implementation is because `Timer` can't support 0 period, so I have to make it run forever.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tony810430 commented on the issue: https://github.com/apache/flink/pull/2925 I think it would not be sub-optimal, because it still need to separate consuming thread and ingesting thread, which emits records to next operator. The reason for this implementation is because `Timer` can't support 0 period, so I have to make it run forever.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/2925

          @tony810430 ok, thanks for your response and patience on the review.
          I'll proceed with reviewing this and testing it against Kinesis either today or tomorrow.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2925 @tony810430 ok, thanks for your response and patience on the review. I'll proceed with reviewing this and testing it against Kinesis either today or tomorrow.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99066645

          — 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);

              • End diff –

          I would not use a "recommend change to configuration" message as warning here, because the user can't really change the setting while the job is still running. An informative message just explaining that fetch took longer than the fetch interval is fine.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99066645 — 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); End diff – I would not use a "recommend change to configuration" message as warning here, because the user can't really change the setting while the job is still running. An informative message just explaining that fetch took longer than the fetch interval is fine.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r95133373

          — Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java —
          @@ -107,27 +116,30 @@ protected ShardConsumer(KinesisDataFetcher<T> fetcherRef,
          this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
          ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
          Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
          +
          + this.error = checkNotNull(error);
          }

          @SuppressWarnings("unchecked")
          @Override
          public void run() {

          • String nextShardItr;
            + String startShardItr;
            + Timer timer = new Timer();
              • End diff –

          It's a good practice to have specified names for the threads created through executor services like this. That'll make it easier to pinpoint any related problems / leaks when debugging.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r95133373 — Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java — @@ -107,27 +116,30 @@ protected ShardConsumer(KinesisDataFetcher<T> fetcherRef, this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty( ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS, Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS))); + + this.error = checkNotNull(error); } @SuppressWarnings("unchecked") @Override public void run() { String nextShardItr; + String startShardItr; + Timer timer = new Timer(); End diff – It's a good practice to have specified names for the threads created through executor services like this. That'll make it easier to pinpoint any related problems / leaks when debugging.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r95134466

          — 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();

              • End diff –

          Calling `cancel` on a Timer doesn't handle any in-progress tasks, it only discards the current scheduled tasks. So for the case where `fetchIntervalMillis` is 0, the forever running fetcher task will not be terminated.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r95134466 — 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(); End diff – Calling `cancel` on a Timer doesn't handle any in-progress tasks, it only discards the current scheduled tasks. So for the case where `fetchIntervalMillis` is 0, the forever running fetcher task will not be terminated.
          Hide
          githubbot ASF GitHub Bot added a comment -

          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?

          Show
          githubbot ASF GitHub Bot added a comment - 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?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99065929

          — 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 {

              • End diff –

          Would be nice to have Javadoc explaining what this task does.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99065929 — 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 { End diff – Would be nice to have Javadoc explaining what this task does.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99067569

          — 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);
            +
            + if (getRecordsResult != null) {
            + // each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
            + List<UserRecord> fetchedRecords = deaggregateRecords(

              • End diff –

          Same here: make `fetchedRecords` reusable?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99067569 — 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); + + if (getRecordsResult != null) { + // each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding + List<UserRecord> fetchedRecords = deaggregateRecords( End diff – Same here: make `fetchedRecords` reusable?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99066877

          — 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);
            +
            + if (getRecordsResult != null) {
            + // 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());
            +
            + for (UserRecord record : fetchedRecords) {
            + boolean notFull = false;
            + while (!notFull) {
            + notFull = userRecordQueue.offer(record);

              • End diff –

          `offer` does not block. If you want to block when adding an element to blocking queue, use `put`.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99066877 — 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); + + if (getRecordsResult != null) { + // 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()); + + for (UserRecord record : fetchedRecords) { + boolean notFull = false; + while (!notFull) { + notFull = userRecordQueue.offer(record); End diff – `offer` does not block. If you want to block when adding an element to blocking queue, use `put`.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99067629

          — Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java —
          @@ -203,6 +288,12 @@ private boolean isRunning()

          { return !Thread.interrupted(); }

          + /** Called by created TimerTask to pass on errors. Only the first thrown error is set.
          — End diff –

          I would simply reference `

          {@link ShardConsumerFetcher}

          ` to be more specific.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99067629 — Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java — @@ -203,6 +288,12 @@ private boolean isRunning() { return !Thread.interrupted(); } + /** Called by created TimerTask to pass on errors. Only the first thrown error is set. — End diff – I would simply reference ` {@link ShardConsumerFetcher} ` to be more specific.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99067395

          — 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) {

              • End diff –

          I think `scheduledExecutionTime ` return the time of the actual execution time, so this should always be false.
          See https://docs.oracle.com/javase/7/docs/api/java/util/TimerTask.html#scheduledExecutionTime().

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99067395 — 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) { End diff – I think `scheduledExecutionTime ` return the time of the actual execution time , so this should always be false. See https://docs.oracle.com/javase/7/docs/api/java/util/TimerTask.html#scheduledExecutionTime( ).
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r95132586

          — Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java —
          @@ -88,6 +96,7 @@ protected ShardConsumer(KinesisDataFetcher<T> fetcherRef,
          Integer subscribedShardStateIndex,
          KinesisStreamShard subscribedShard,
          SequenceNumber lastSequenceNum,
          + AtomicReference<Throwable> error,
          — End diff –

          I don't think you need to add this constructor argument here, because it isn't used in the tests, correct?

          This protected constructor exists for testing purposes. For example, in the tests, we mock a `KinesisProxyInterface` and inject it into a `ShardConsumer` under test through this constructor.

          On the other hand, it'll be good to add tests related to error handling across the new threads, in which case this constructor change can be left as is.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r95132586 — Diff: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java — @@ -88,6 +96,7 @@ protected ShardConsumer(KinesisDataFetcher<T> fetcherRef, Integer subscribedShardStateIndex, KinesisStreamShard subscribedShard, SequenceNumber lastSequenceNum, + AtomicReference<Throwable> error, — End diff – I don't think you need to add this constructor argument here, because it isn't used in the tests, correct? This protected constructor exists for testing purposes. For example, in the tests, we mock a `KinesisProxyInterface` and inject it into a `ShardConsumer` under test through this constructor. On the other hand, it'll be good to add tests related to error handling across the new threads, in which case this constructor change can be left as is.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99065974

          — 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.*/

              • End diff –

          nit: Missing space before "*/" at the end

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99065974 — 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.*/ End diff – nit: Missing space before "*/" at the end
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tony810430 commented on a diff in the pull request:

          https://github.com/apache/flink/pull/2925#discussion_r99117155

          — 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) {

              • End diff –

          The implementation for `scheduledExecutionTime` is
          ```
          public long scheduledExecutionTime() {
          synchronized(lock)

          { return (period < 0 ? nextExecutionTime + period : nextExecutionTime - period); }

          }
          ```
          and `nextExecutionTime` is updated by `nextExecutionTime + period`.

          In `scheduleAtFixedRate` mode, `scheduledExecutionTime` will actually return the expected execution time, so I would use it to determine if the task was delayed.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tony810430 commented on a diff in the pull request: https://github.com/apache/flink/pull/2925#discussion_r99117155 — 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) { End diff – The implementation for `scheduledExecutionTime` is ``` public long scheduledExecutionTime() { synchronized(lock) { return (period < 0 ? nextExecutionTime + period : nextExecutionTime - period); } } ``` and `nextExecutionTime` is updated by `nextExecutionTime + period`. In `scheduleAtFixedRate` mode, `scheduledExecutionTime` will actually return the expected execution time, so I would use it to determine if the task was delayed.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tony810430 commented on the issue:

          https://github.com/apache/flink/pull/2925

          Hi, @tzulitai

          I think there is no better way if trying to use `Timer` to deal with `0` fetch interval. Even though I implement my own `Timer` scheduler, it just move the `runForever` inside from `ShardConsumerTask`. Because fetch interval is `0`, there is no way to get the next expected execution time and the solution would be running the function forever.

          I think a better solution is defined non-positive fetch interval differs from positive fetch interval and would be executed in another way in the document. The implementation will be two parts: the original way remains the same without `runForever` in `ShardConsumerFetcherTask` and the other is for non-positive fetch interval implemented by using a Thread running `ShardConsumerFetcherTask::run()` forever.

          It is more reasonable for me to distinguish these two configuration and implement them by using two ways in `ShardConsumer::run()`. What do you think?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tony810430 commented on the issue: https://github.com/apache/flink/pull/2925 Hi, @tzulitai I think there is no better way if trying to use `Timer` to deal with `0` fetch interval. Even though I implement my own `Timer` scheduler, it just move the `runForever` inside from `ShardConsumerTask`. Because fetch interval is `0`, there is no way to get the next expected execution time and the solution would be running the function forever. I think a better solution is defined non-positive fetch interval differs from positive fetch interval and would be executed in another way in the document. The implementation will be two parts: the original way remains the same without `runForever` in `ShardConsumerFetcherTask` and the other is for non-positive fetch interval implemented by using a Thread running `ShardConsumerFetcherTask::run()` forever. It is more reasonable for me to distinguish these two configuration and implement them by using two ways in `ShardConsumer::run()`. What do you think?

            People

            • Assignee:
              tonywei Wei-Che Wei
              Reporter:
              tzulitai Tzu-Li (Gordon) Tai
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:

                Development