You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tzulitai <gi...@git.apache.org> on 2017/02/02 08:08:28 UTC

[GitHub] flink pull request #2925: [FLINK-4574] [kinesis] Strengthen fetch interval i...

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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---