You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/07/13 09:06:27 UTC

[GitHub] [flink] dannycranmer opened a new pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

dannycranmer opened a new pull request #12881:
URL: https://github.com/apache/flink/pull/12881


   ## What is the purpose of the change
   
   This is the second milestone of [FLIP-128](https://cwiki.apache.org/confluence/display/FLINK/FLIP-128%3A+Enhanced+Fan+Out+for+AWS+Kinesis+Consumers) to add EFO support to the `FlinkKinesisConsumer`. The consumption mechanism in `ShardConsumer` has been abstracted to an interface and implemented to retain existing behaviour. An EFO implementation will be added in a future contribution. This change should not introduce any functional differences.
   
   ## Note
   This PR is blocked by:
   - Test coverage improvements:
     - [JIRA issue](https://issues.apache.org/jira/browse/FLINK-18483)
     - [Pull request](https://github.com/apache/flink/pull/12850)
   
   ## Brief change log
   
   - `RecordPublisher`
     - Added a new interface used to consume records from Kinesis and supply them to Flink
   - `PollingRecordPublisher` 
     - The existing consumption mechanism using `AWS::GetRecords` and `AWS::GetShardIterator` has been refactored into an implementation of `RecordPublisher`. A factory has been added to create instances
     - Configuration that was deserialised in the constructor has been moved to a class `PollingRecordPublisherConfiguration`
   - `AdaptivePollingRecordPublisher`
     - When a user supplies `useAdaptiveReads` a special `RecordPublisher` is created that adds adaptive loop delay and batch size. This functionality has been moved from `ShardConsumer`, but now has it's own implementation that extends `PollingRecordPublisher`
   -  `StartingPosition`
      - A new object has been created to encapsulate starting sequence number/iterator type. A starting position can be created from a `SentinelSequenceNumber` 
   - `ShardMetricsReporter`
     - Metric reporting has been split to allow arbitrary metrics to be registered by `RecordPublisher` implementations
     - `ShardConsumer` reports general metrics using `ShardConsumerMetricsReporter`
     - `PollingRecordPublisher` reports metrics using `PollingRecordPublisherMetricsReporter`
   - ` ShardConsumer`
     - Refactored to accept a `RecordPublisher` and not depend on a `KinesisProxyInterface`
     - Restarting from aggregated sequence number logic moved to support generic `RecordPublishers`:
       - Before: At startup a single record was downloaded and already processed subsequences were discarded
       - After: During consumption any already processed subsequences are discarded
       - Note: This does add a small overhead to the continuous consumption, but simplifies `RecordPublishers`. An alternative could be to code the `ShardConsumer` to handle restarting from aggregated records via a standard `KinesisProxy`
   
   ## Verifying this change
   
   This change is already covered by existing tests, such as:
   - `ShardConsumerTest`
   
   A preceding pull request was submitted to increase test coverage before performing the refactor:
   - https://github.com/apache/flink/pull/12850
   
   This change added tests and can be verified as follows:
   - `RecordBatch` -> `RecordBatchTest`
   - `PollingRecordPublisherConfiguration` -> `PollingRecordPublisherConfigurationTest`
   - `PollingRecordPublisherFactory` -> `PollingRecordPublisherFactoryTest`
   - `PollingRecordPublisher` -> `PollingRecordPublisherTest`
   - `PollingRecordPublisherMetricsReporter` -> `PollingRecordPublisherMetricsReporterTest`
   - `ShardConsumerMetricsReporter` -> `ShardConsumerMetricsReporterTest`
   - `StartingPosition` -> `StartingPositionTest`
   
   I have deployed Flink applications locally and on AWS KDA-Java and verified consumption.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no (dependency change is covered by FLINK-18483)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): yes (added check for filtering de-aggregated subsequence numbers `ShardConsumer::filterDeaggregatedRecord`)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: yes (restarting consumption from sequence number)
     - The S3 file system connector: no
   
   ## Documentation
   
     - Does this pull request introduce a new feature? no
     - If yes, how is the feature documented? not applicable
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] xiaolong-sn commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
xiaolong-sn commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r454251135



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyMarker.java
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * A marker interface for generic Kinesis Proxy.
+ */
+@Internal

Review comment:
       What does this interface represent besides the existed KinesisProxyInterface?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -379,49 +205,18 @@ private void deserializeRecordForCollectionAndUpdateState(UserRecord record)
 	}
 
 	/**
-	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
-	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
-	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
-	 * be used for the next call to this method.
-	 *
-	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
-	 * fully collected with {@link ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
-	 * {@link ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
-	 * incorrect shard iteration if the iterator had to be refreshed.
+	 * Filters out aggregated records that have previously been processed.
+	 * This method is to support restarting from a partially consumed aggregated sequence number.
 	 *
-	 * @param shardItr shard iterator to use
-	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
-	 * @return get records result
-	 * @throws InterruptedException
+	 * @param record the record to filter
+	 * @return {@code true} if the record should be retained
 	 */
-	private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) throws Exception {
-		GetRecordsResult getRecordsResult = null;
-		while (getRecordsResult == null) {
-			try {
-				getRecordsResult = kinesis.getRecords(shardItr, maxNumberOfRecords);
-
-				// Update millis behind latest so it gets reported by the millisBehindLatest gauge
-				Long millisBehindLatest = getRecordsResult.getMillisBehindLatest();
-				if (millisBehindLatest != null) {
-					shardMetricsReporter.setMillisBehindLatest(millisBehindLatest);
-				}
-			} catch (ExpiredIteratorException eiEx) {
-				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
-					" refreshing the iterator ...", shardItr, subscribedShard);
-
-				shardItr = getShardIterator(lastSequenceNum);
-
-				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
-				if (fetchIntervalMillis != 0) {
-					Thread.sleep(fetchIntervalMillis);
-				}
-			}
+	private boolean filterDeaggregatedRecord(final UserRecord record) {
+		if (lastSequenceNum.isAggregated()) {

Review comment:
       What does this function means? 

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(
+			final StreamShardHandle subscribedShard,
+			final PollingRecordPublisherMetricsReporter metricsReporter,
+			final KinesisProxyInterface kinesisProxy,
+			final int maxNumberOfRecordsPerFetch,
+			final long expiredIteratorBackoffMillis) {
+		this.subscribedShard = subscribedShard;
+		this.metricsReporter = metricsReporter;
+		this.kinesisProxy = kinesisProxy;
+		this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+		this.expiredIteratorBackoffMillis = expiredIteratorBackoffMillis;
+	}
+
+	@Override
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer) throws InterruptedException {
+		return run(startingPosition, consumer, maxNumberOfRecordsPerFetch);
+	}
+
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer, int maxNumberOfRecords) throws InterruptedException {
+		if (nextShardItr == null) {
+			nextShardItr = getShardIterator(startingPosition);
+		}
+
+		if (nextShardItr == null) {
+			return COMPLETE;
+		}
+
+		metricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecords);
+
+		GetRecordsResult result = getRecords(nextShardItr, startingPosition, maxNumberOfRecords);
+
+		consumer.accept(new RecordBatch(result.getRecords(), subscribedShard, result.getMillisBehindLatest()));
+
+		nextShardItr = result.getNextShardIterator();
+		return nextShardItr == null ? COMPLETE : INCOMPLETE;
+	}
+
+	/**
+	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
+	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
+	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
+	 * be used for the next call to this method.
+	 *
+	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
+	 * fully collected with {@code ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
+	 * {@code ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
+	 * incorrect shard iteration if the iterator had to be refreshed.
+	 *
+	 * @param shardItr shard iterator to use
+	 * @param startingPosition the position in the stream in which to consume from
+	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
+	 * @return get records result
+	 */
+	private GetRecordsResult getRecords(String shardItr, final StartingPosition startingPosition, int maxNumberOfRecords) throws InterruptedException {
+		GetRecordsResult getRecordsResult = null;
+		while (getRecordsResult == null) {
+			try {
+				getRecordsResult = kinesisProxy.getRecords(shardItr, maxNumberOfRecords);
+			} catch (ExpiredIteratorException | InterruptedException eiEx) {

Review comment:
       Should we also consider other recoverable exceptions like `ProvisionedThroughputExceededException` here?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
##########
@@ -0,0 +1,106 @@
+/*

Review comment:
       This class looks quite like the `SentinelSequenceNumber ` class, is there any possibility to combine those two classes?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812) Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) 
   * e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468867443



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
-
-	// AWS Kinesis has a read limit of 2 Mb/sec
-	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
-	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
-
 	private final KinesisDeserializationSchema<T> deserializer;
 
-	private final KinesisProxyInterface kinesis;
-
 	private final int subscribedShardStateIndex;
 
 	private final KinesisDataFetcher<T> fetcherRef;
 
 	private final StreamShardHandle subscribedShard;
 
-	private int maxNumberOfRecordsPerFetch;
-	private final long fetchIntervalMillis;
-	private final boolean useAdaptiveReads;
+	private final ShardConsumerMetricsReporter shardConsumerMetricsReporter;
 
-	private final ShardMetricsReporter shardMetricsReporter;
+	private StartingPosition startingPosition;
 
 	private SequenceNumber lastSequenceNum;
 
-	private Date initTimestamp;
+	private final RecordPublisher recordPublisher;
 
 	/**
 	 * Creates a shard consumer.
 	 *
 	 * @param fetcherRef reference to the owning fetcher
+	 * @param recordPublisher the record publisher used to read records from kinesis
 	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
 	 * @param subscribedShard the shard this consumer is subscribed to
 	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 * @param kinesis the proxy instance to interact with Kinesis
-	 * @param shardMetricsReporter the reporter to report metrics to
+	 * @param shardConsumerMetricsReporter the reporter to report metrics to
+	 * @param shardDeserializer used to deserialize incoming records
 	 */
 	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+						RecordPublisher recordPublisher,
 						Integer subscribedShardStateIndex,
 						StreamShardHandle subscribedShard,
 						SequenceNumber lastSequenceNum,
-						KinesisProxyInterface kinesis,
-						ShardMetricsReporter shardMetricsReporter,
+						ShardConsumerMetricsReporter shardConsumerMetricsReporter,
 						KinesisDeserializationSchema<T> shardDeserializer) {
 		this.fetcherRef = checkNotNull(fetcherRef);
+		this.recordPublisher = checkNotNull(recordPublisher);
 		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
 		this.subscribedShard = checkNotNull(subscribedShard);
+		this.shardConsumerMetricsReporter = checkNotNull(shardConsumerMetricsReporter);
 		this.lastSequenceNum = checkNotNull(lastSequenceNum);
 
-		this.shardMetricsReporter = checkNotNull(shardMetricsReporter);
-
 		checkArgument(
 			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
 			"Should not start a ShardConsumer if the shard has already been completely read.");
 
 		this.deserializer = shardDeserializer;
 
 		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
-		this.kinesis = kinesis;
-		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
-			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
-		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
-			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
-		this.useAdaptiveReads = Boolean.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS,
-			Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS)));
-
-		if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+
+		if (lastSequenceNum.equals(SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+			Date initTimestamp;
 			String timestamp = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP);
 
 			try {
 				String format = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT,
 					ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT);
 				SimpleDateFormat customDateFormat = new SimpleDateFormat(format);
-				this.initTimestamp = customDateFormat.parse(timestamp);
+				initTimestamp = customDateFormat.parse(timestamp);
 			} catch (IllegalArgumentException | NullPointerException exception) {
 				throw new IllegalArgumentException(exception);
 			} catch (ParseException exception) {
-				this.initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
+				initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
 			}
-		} else {
-			this.initTimestamp = null;
-		}
-	}
-
-	/**
-	 * Returns a shard iterator for the given {@link SequenceNumber}.
-	 *
-	 * @return shard iterator
-	 * @throws Exception
-	 */
-	protected String getShardIterator(SequenceNumber sequenceNumber) throws Exception {
 
-		if (isSentinelSequenceNumber(sequenceNumber)) {
-			return getShardIteratorForSentinel(sequenceNumber);
+			startingPosition = StartingPosition.fromTimestamp(initTimestamp);
 		} else {
-			// we will be starting from an actual sequence number (due to restore from failure).
-			return getShardIteratorForRealSequenceNumber(sequenceNumber);
+			startingPosition = StartingPosition.restartFromSequenceNumber(checkNotNull(lastSequenceNum));
 		}
 	}
 
-	protected String getShardIteratorForSentinel(SequenceNumber sentinelSequenceNumber) throws InterruptedException {
-		String nextShardItr;
-
-		if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) {
-			// if the shard is already closed, there will be no latest next record to get for this shard
-			if (subscribedShard.isClosed()) {
-				nextShardItr = null;
-			} else {
-				nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.LATEST.toString(), null);
-			}
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) {
-			nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.TRIM_HORIZON.toString(), null);
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-			nextShardItr = null;
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
-			nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AT_TIMESTAMP.toString(), initTimestamp);
-		} else {
-			throw new RuntimeException("Unknown sentinel type: " + sentinelSequenceNumber);
-		}
-
-		return nextShardItr;
-	}
-
-	protected String getShardIteratorForRealSequenceNumber(SequenceNumber sequenceNumber)
-			throws Exception {
-
-		// if the last sequence number refers to an aggregated record, we need to clean up any dangling sub-records
-		// from the last aggregated record; otherwise, we can simply start iterating from the record right after.
-
-		if (sequenceNumber.isAggregated()) {
-			return getShardIteratorForAggregatedSequenceNumber(sequenceNumber);
-		} else {
-			// the last record was non-aggregated, so we can simply start from the next record
-			return kinesis.getShardIterator(
-					subscribedShard,
-					ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(),
-					sequenceNumber.getSequenceNumber());
-		}
-	}
-
-	protected String getShardIteratorForAggregatedSequenceNumber(SequenceNumber sequenceNumber)
-			throws Exception {
-
-		String itrForLastAggregatedRecord =
-				kinesis.getShardIterator(
-						subscribedShard,
-						ShardIteratorType.AT_SEQUENCE_NUMBER.toString(),
-						sequenceNumber.getSequenceNumber());
-
-		// get only the last aggregated record
-		GetRecordsResult getRecordsResult = getRecords(itrForLastAggregatedRecord, 1);
-
-		List<UserRecord> fetchedRecords = deaggregateRecords(
-				getRecordsResult.getRecords(),
-				subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-				subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-		long lastSubSequenceNum = sequenceNumber.getSubSequenceNumber();
-		for (UserRecord record : fetchedRecords) {
-			// we have found a dangling sub-record if it has a larger subsequence number
-			// than our last sequence number; if so, collect the record and update state
-			if (record.getSubSequenceNumber() > lastSubSequenceNum) {
-				deserializeRecordForCollectionAndUpdateState(record);
-			}
-		}
-
-		return getRecordsResult.getNextShardIterator();
-	}
-
-	@SuppressWarnings("unchecked")
 	@Override
 	public void run() {
 		try {
-			String nextShardItr = getShardIterator(lastSequenceNum);
+			while (isRunning()) {
+				final RecordPublisherRunResult result = recordPublisher.run(startingPosition, batch -> {
+					for (UserRecord userRecord : batch.getDeaggregatedRecords()) {
+						if (filterDeaggregatedRecord(userRecord)) {
+							deserializeRecordForCollectionAndUpdateState(userRecord);
+						}
+					}
 
-			long processingStartTimeNanos = System.nanoTime();
+					shardConsumerMetricsReporter.setAverageRecordSizeBytes(batch.getAverageRecordSizeBytes());
+					shardConsumerMetricsReporter.setNumberOfAggregatedRecords(batch.getAggregatedRecordSize());
+					shardConsumerMetricsReporter.setNumberOfDeaggregatedRecords(batch.getDeaggregatedRecordSize());
+					ofNullable(batch.getMillisBehindLatest()).ifPresent(shardConsumerMetricsReporter::setMillisBehindLatest);
+				});
 
-			while (isRunning()) {
-				if (nextShardItr == null) {
+				if (result == COMPLETE) {
 					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 {
-					shardMetricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecordsPerFetch);
-					GetRecordsResult getRecordsResult = getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
-
-					List<Record> aggregatedRecords = getRecordsResult.getRecords();
-					int numberOfAggregatedRecords = aggregatedRecords.size();
-					shardMetricsReporter.setNumberOfAggregatedRecords(numberOfAggregatedRecords);
-
-					// each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
-					List<UserRecord> fetchedRecords = deaggregateRecords(
-						aggregatedRecords,
-						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-					long recordBatchSizeBytes = 0L;
-					for (UserRecord record : fetchedRecords) {
-						recordBatchSizeBytes += record.getData().remaining();
-						deserializeRecordForCollectionAndUpdateState(record);
-					}
-
-					int numberOfDeaggregatedRecords = fetchedRecords.size();
-					shardMetricsReporter.setNumberOfDeaggregatedRecords(numberOfDeaggregatedRecords);
-
-					nextShardItr = getRecordsResult.getNextShardIterator();
-
-					long adjustmentEndTimeNanos = adjustRunLoopFrequency(processingStartTimeNanos, System.nanoTime());
-					long runLoopTimeNanos = adjustmentEndTimeNanos - processingStartTimeNanos;
-					maxNumberOfRecordsPerFetch = adaptRecordsToRead(runLoopTimeNanos, fetchedRecords.size(), recordBatchSizeBytes, maxNumberOfRecordsPerFetch);
-					shardMetricsReporter.setRunLoopTimeNanos(runLoopTimeNanos);
-					processingStartTimeNanos = adjustmentEndTimeNanos; // for next time through the loop
+					startingPosition = StartingPosition.continueFromSequenceNumber(lastSequenceNum);

Review comment:
       See other comment regarding `StartingPosition`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468859151



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import java.util.function.Consumer;
+
+/**
+ * An adaptive record publisher to add a dynamic loop delay and batch read size for {@link PollingRecordPublisher}.
+ * Kinesis Streams have quotas on the transactions per second, and throughout. This class attempts to balance
+ * quotas and mitigate back off errors.
+ */
+@Internal
+public class AdaptivePollingRecordPublisher extends PollingRecordPublisher {

Review comment:
       Thanks! :D




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tzulitai commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
tzulitai commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468417898



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(

Review comment:
       constructor can be package-private

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(
+			final StreamShardHandle subscribedShard,
+			final PollingRecordPublisherMetricsReporter metricsReporter,
+			final KinesisProxyInterface kinesisProxy,
+			final int maxNumberOfRecordsPerFetch,
+			final long expiredIteratorBackoffMillis) {
+		this.subscribedShard = subscribedShard;
+		this.metricsReporter = metricsReporter;
+		this.kinesisProxy = kinesisProxy;
+		this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+		this.expiredIteratorBackoffMillis = expiredIteratorBackoffMillis;
+	}
+
+	@Override
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer) throws InterruptedException {
+		return run(startingPosition, consumer, maxNumberOfRecordsPerFetch);
+	}
+
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer, int maxNumberOfRecords) throws InterruptedException {
+		if (nextShardItr == null) {
+			nextShardItr = getShardIterator(startingPosition);
+		}
+
+		if (nextShardItr == null) {
+			return COMPLETE;
+		}
+
+		metricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecords);
+
+		GetRecordsResult result = getRecords(nextShardItr, startingPosition, maxNumberOfRecords);
+
+		consumer.accept(new RecordBatch(result.getRecords(), subscribedShard, result.getMillisBehindLatest()));
+
+		nextShardItr = result.getNextShardIterator();
+		return nextShardItr == null ? COMPLETE : INCOMPLETE;
+	}
+
+	/**
+	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
+	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
+	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
+	 * be used for the next call to this method.
+	 *
+	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
+	 * fully collected with {@code ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
+	 * {@code ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
+	 * incorrect shard iteration if the iterator had to be refreshed.
+	 *
+	 * @param shardItr shard iterator to use
+	 * @param startingPosition the position in the stream in which to consume from
+	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
+	 * @return get records result
+	 */
+	private GetRecordsResult getRecords(String shardItr, final StartingPosition startingPosition, int maxNumberOfRecords) throws InterruptedException {
+		GetRecordsResult getRecordsResult = null;
+		while (getRecordsResult == null) {
+			try {
+				getRecordsResult = kinesisProxy.getRecords(shardItr, maxNumberOfRecords);
+			} catch (ExpiredIteratorException eiEx) {
+				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
+					" refreshing the iterator ...", shardItr, subscribedShard);
+
+				shardItr = getShardIterator(startingPosition);
+
+				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
+				if (expiredIteratorBackoffMillis != 0) {
+					Thread.sleep(expiredIteratorBackoffMillis);
+				}
+			}
+		}
+		return getRecordsResult;
+	}
+
+	/**
+	 * Returns a shard iterator for the given {@link SequenceNumber}.
+	 *
+	 * @return shard iterator
+	 */
+	@Nullable
+	private String getShardIterator(final StartingPosition startingPosition) throws InterruptedException {
+		if (startingPosition.getShardIteratorType() == LATEST && subscribedShard.isClosed()) {
+			return null;
+		}
+
+		return kinesisProxy.getShardIterator(

Review comment:
       Double checking for my understanding:
   Does this return `null` for an out-of-bounds position?
   Say, the given timestamp is out of bounds or the sequence number is non-existent.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(
+			final StreamShardHandle subscribedShard,
+			final PollingRecordPublisherMetricsReporter metricsReporter,
+			final KinesisProxyInterface kinesisProxy,
+			final int maxNumberOfRecordsPerFetch,
+			final long expiredIteratorBackoffMillis) {
+		this.subscribedShard = subscribedShard;

Review comment:
       Could you add simple sanity checks on these arguments, such as `Preconditions.checkNonNull`?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {

Review comment:
       With the record polling refactored out, could you rephrase the class-level Javadoc of this class?
   Essentially this is now a thread that just subscribes to a `RecordPublisher` which publishes records from a single Kinesis shard.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -379,49 +206,18 @@ private void deserializeRecordForCollectionAndUpdateState(UserRecord record)
 	}
 
 	/**
-	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
-	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
-	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
-	 * be used for the next call to this method.
+	 * Filters out aggregated records that have previously been processed.
+	 * This method is to support restarting from a partially consumed aggregated sequence number.
 	 *
-	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
-	 * fully collected with {@link ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
-	 * {@link ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
-	 * incorrect shard iteration if the iterator had to be refreshed.
-	 *
-	 * @param shardItr shard iterator to use
-	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
-	 * @return get records result
-	 * @throws InterruptedException
+	 * @param record the record to filter
+	 * @return {@code true} if the record should be retained
 	 */
-	private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) throws Exception {
-		GetRecordsResult getRecordsResult = null;
-		while (getRecordsResult == null) {
-			try {
-				getRecordsResult = kinesis.getRecords(shardItr, maxNumberOfRecords);
-
-				// Update millis behind latest so it gets reported by the millisBehindLatest gauge
-				Long millisBehindLatest = getRecordsResult.getMillisBehindLatest();
-				if (millisBehindLatest != null) {
-					shardMetricsReporter.setMillisBehindLatest(millisBehindLatest);
-				}
-			} catch (ExpiredIteratorException eiEx) {
-				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
-					" refreshing the iterator ...", shardItr, subscribedShard);
-
-				shardItr = getShardIterator(lastSequenceNum);
-
-				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
-				if (fetchIntervalMillis != 0) {
-					Thread.sleep(fetchIntervalMillis);
-				}
-			}
+	private boolean filterDeaggregatedRecord(final UserRecord record) {
+		if (lastSequenceNum.isAggregated()) {
+			return !record.getSequenceNumber().equals(lastSequenceNum.getSequenceNumber()) ||
+				record.getSubSequenceNumber() > lastSequenceNum.getSubSequenceNumber();
 		}
-		return getRecordsResult;
-	}
 
-	@SuppressWarnings("unchecked")
-	protected static List<UserRecord> deaggregateRecords(List<Record> records, String startingHashKey, String endingHashKey) {
-		return UserRecord.deaggregate(records, new BigInteger(startingHashKey), new BigInteger(endingHashKey));
+		return true;

Review comment:
       Just for easier code readability, can we rewrite this as:
   
   ```
   if (!lastSequenceNum.isAggregated()) {
       return true;
   }
   
   return !record.getSequenceNumber().equals(lastSequenceNum.getSequenceNumber()) ||
   				record.getSubSequenceNumber() > lastSequenceNum.getSubSequenceNumber();
   ```
   
   I've found early returns always helps with understanding these :)

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordBatch.java
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
+import com.amazonaws.services.kinesis.model.Record;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.List;
+
+/**
+ * A batch of UserRecords received from Kinesis.
+ * Input records are de-aggregated using KCL 1.x library.
+ * It is expected that AWS SDK v2.x messages are converted to KCL 1.x {@link UserRecord}.
+ */
+public class RecordBatch {
+
+	private final int aggregatedRecordSize;
+
+	private final List<UserRecord> deaggregatedRecords;
+
+	private final long totalSizeInBytes;
+
+	private final Long millisBehindLatest;
+
+	public RecordBatch(final List<Record> records, final StreamShardHandle subscribedShard, final Long millisBehindLatest) {
+		this.aggregatedRecordSize = records.size();

Review comment:
       Argument sanity / precondition checks.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import java.util.function.Consumer;
+
+/**
+ * An adaptive record publisher to add a dynamic loop delay and batch read size for {@link PollingRecordPublisher}.
+ * Kinesis Streams have quotas on the transactions per second, and throughout. This class attempts to balance
+ * quotas and mitigate back off errors.
+ */
+@Internal
+public class AdaptivePollingRecordPublisher extends PollingRecordPublisher {
+	// AWS Kinesis has a read limit of 2 Mb/sec
+	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
+	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
+
+	private int lastRecordBatchSize = 0;
+
+	private long lastRecordBatchSizeInBytes = 0;
+
+	private long processingStartTimeNanos = System.nanoTime();
+
+	private int maxNumberOfRecordsPerFetch;
+
+	private final long fetchIntervalMillis;
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	public AdaptivePollingRecordPublisher(

Review comment:
       constructor can be package-private

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisher.java
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+
+import java.util.function.Consumer;
+
+/**
+ * A {@code RecordPublisher} will consume records from an external stream and deliver them to the registered subscriber.
+ */
+@Internal
+public interface RecordPublisher {
+
+	/**
+	 * Run the record publisher. Records will be consumed from the stream and published to the consumer.
+	 * The number of batches retrieved by a single invocation will vary based on implementation.
+	 *
+	 * @param startingPosition the position in the stream from which to consume
+	 * @param recordConsumer the record consumer in which to output records
+	 * @return a status enum to represent whether a shard has been fully consumed
+	 * @throws InterruptedException
+	 */
+	RecordPublisherRunResult run(StartingPosition startingPosition, Consumer<RecordBatch> recordConsumer) throws InterruptedException;

Review comment:
       I found that the argument `StartingPosition` was slightly confusing to be passed in here, since from the implementations it seem like this `startingPosition` is only ever used on the first getRecords call, but with this interface we would be passing in the starting position on every invocation.
   
   It's also slightly more error-prone for the call site of `RecordPublisher`s, imagine if the call site continued to invoke `run` with a starting position, after the previous attempt already returned `COMPLETE` as a result.
   
   As a food for thought: having the `StartingPosition` passed in via the `RecordPublisherFactory` could potentially be a better option?

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+
+import javax.annotation.Nullable;
+
+import java.util.Date;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.TRIM_HORIZON;
+import static org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber.isSentinelSequenceNumber;
+
+/**
+ * The position in which to start consuming from a stream.
+ */
+public class StartingPosition {

Review comment:
       Mark with `@Internal`

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
-
-	// AWS Kinesis has a read limit of 2 Mb/sec
-	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
-	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
-
 	private final KinesisDeserializationSchema<T> deserializer;
 
-	private final KinesisProxyInterface kinesis;
-
 	private final int subscribedShardStateIndex;
 
 	private final KinesisDataFetcher<T> fetcherRef;
 
 	private final StreamShardHandle subscribedShard;
 
-	private int maxNumberOfRecordsPerFetch;
-	private final long fetchIntervalMillis;
-	private final boolean useAdaptiveReads;
+	private final ShardConsumerMetricsReporter shardConsumerMetricsReporter;
 
-	private final ShardMetricsReporter shardMetricsReporter;
+	private StartingPosition startingPosition;
 
 	private SequenceNumber lastSequenceNum;
 
-	private Date initTimestamp;
+	private final RecordPublisher recordPublisher;
 
 	/**
 	 * Creates a shard consumer.
 	 *
 	 * @param fetcherRef reference to the owning fetcher
+	 * @param recordPublisher the record publisher used to read records from kinesis
 	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
 	 * @param subscribedShard the shard this consumer is subscribed to
 	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 * @param kinesis the proxy instance to interact with Kinesis
-	 * @param shardMetricsReporter the reporter to report metrics to
+	 * @param shardConsumerMetricsReporter the reporter to report metrics to
+	 * @param shardDeserializer used to deserialize incoming records
 	 */
 	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+						RecordPublisher recordPublisher,
 						Integer subscribedShardStateIndex,
 						StreamShardHandle subscribedShard,
 						SequenceNumber lastSequenceNum,
-						KinesisProxyInterface kinesis,
-						ShardMetricsReporter shardMetricsReporter,
+						ShardConsumerMetricsReporter shardConsumerMetricsReporter,
 						KinesisDeserializationSchema<T> shardDeserializer) {
 		this.fetcherRef = checkNotNull(fetcherRef);
+		this.recordPublisher = checkNotNull(recordPublisher);
 		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
 		this.subscribedShard = checkNotNull(subscribedShard);
+		this.shardConsumerMetricsReporter = checkNotNull(shardConsumerMetricsReporter);
 		this.lastSequenceNum = checkNotNull(lastSequenceNum);
 
-		this.shardMetricsReporter = checkNotNull(shardMetricsReporter);
-
 		checkArgument(
 			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
 			"Should not start a ShardConsumer if the shard has already been completely read.");
 
 		this.deserializer = shardDeserializer;
 
 		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
-		this.kinesis = kinesis;
-		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
-			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
-		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
-			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
-		this.useAdaptiveReads = Boolean.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS,
-			Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS)));
-
-		if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+
+		if (lastSequenceNum.equals(SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+			Date initTimestamp;
 			String timestamp = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP);
 
 			try {
 				String format = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT,
 					ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT);
 				SimpleDateFormat customDateFormat = new SimpleDateFormat(format);
-				this.initTimestamp = customDateFormat.parse(timestamp);
+				initTimestamp = customDateFormat.parse(timestamp);
 			} catch (IllegalArgumentException | NullPointerException exception) {
 				throw new IllegalArgumentException(exception);
 			} catch (ParseException exception) {
-				this.initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
+				initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
 			}
-		} else {
-			this.initTimestamp = null;
-		}
-	}
-
-	/**
-	 * Returns a shard iterator for the given {@link SequenceNumber}.
-	 *
-	 * @return shard iterator
-	 * @throws Exception
-	 */
-	protected String getShardIterator(SequenceNumber sequenceNumber) throws Exception {
 
-		if (isSentinelSequenceNumber(sequenceNumber)) {
-			return getShardIteratorForSentinel(sequenceNumber);
+			startingPosition = StartingPosition.fromTimestamp(initTimestamp);
 		} else {
-			// we will be starting from an actual sequence number (due to restore from failure).
-			return getShardIteratorForRealSequenceNumber(sequenceNumber);
+			startingPosition = StartingPosition.restartFromSequenceNumber(checkNotNull(lastSequenceNum));
 		}
 	}
 
-	protected String getShardIteratorForSentinel(SequenceNumber sentinelSequenceNumber) throws InterruptedException {
-		String nextShardItr;
-
-		if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) {
-			// if the shard is already closed, there will be no latest next record to get for this shard
-			if (subscribedShard.isClosed()) {
-				nextShardItr = null;
-			} else {
-				nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.LATEST.toString(), null);
-			}
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) {
-			nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.TRIM_HORIZON.toString(), null);
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-			nextShardItr = null;
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
-			nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AT_TIMESTAMP.toString(), initTimestamp);
-		} else {
-			throw new RuntimeException("Unknown sentinel type: " + sentinelSequenceNumber);
-		}
-
-		return nextShardItr;
-	}
-
-	protected String getShardIteratorForRealSequenceNumber(SequenceNumber sequenceNumber)
-			throws Exception {
-
-		// if the last sequence number refers to an aggregated record, we need to clean up any dangling sub-records
-		// from the last aggregated record; otherwise, we can simply start iterating from the record right after.
-
-		if (sequenceNumber.isAggregated()) {
-			return getShardIteratorForAggregatedSequenceNumber(sequenceNumber);
-		} else {
-			// the last record was non-aggregated, so we can simply start from the next record
-			return kinesis.getShardIterator(
-					subscribedShard,
-					ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(),
-					sequenceNumber.getSequenceNumber());
-		}
-	}
-
-	protected String getShardIteratorForAggregatedSequenceNumber(SequenceNumber sequenceNumber)
-			throws Exception {
-
-		String itrForLastAggregatedRecord =
-				kinesis.getShardIterator(
-						subscribedShard,
-						ShardIteratorType.AT_SEQUENCE_NUMBER.toString(),
-						sequenceNumber.getSequenceNumber());
-
-		// get only the last aggregated record
-		GetRecordsResult getRecordsResult = getRecords(itrForLastAggregatedRecord, 1);
-
-		List<UserRecord> fetchedRecords = deaggregateRecords(
-				getRecordsResult.getRecords(),
-				subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-				subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-		long lastSubSequenceNum = sequenceNumber.getSubSequenceNumber();
-		for (UserRecord record : fetchedRecords) {
-			// we have found a dangling sub-record if it has a larger subsequence number
-			// than our last sequence number; if so, collect the record and update state
-			if (record.getSubSequenceNumber() > lastSubSequenceNum) {
-				deserializeRecordForCollectionAndUpdateState(record);
-			}
-		}
-
-		return getRecordsResult.getNextShardIterator();
-	}
-
-	@SuppressWarnings("unchecked")
 	@Override
 	public void run() {
 		try {
-			String nextShardItr = getShardIterator(lastSequenceNum);
+			while (isRunning()) {
+				final RecordPublisherRunResult result = recordPublisher.run(startingPosition, batch -> {
+					for (UserRecord userRecord : batch.getDeaggregatedRecords()) {
+						if (filterDeaggregatedRecord(userRecord)) {
+							deserializeRecordForCollectionAndUpdateState(userRecord);
+						}
+					}
 
-			long processingStartTimeNanos = System.nanoTime();
+					shardConsumerMetricsReporter.setAverageRecordSizeBytes(batch.getAverageRecordSizeBytes());
+					shardConsumerMetricsReporter.setNumberOfAggregatedRecords(batch.getAggregatedRecordSize());
+					shardConsumerMetricsReporter.setNumberOfDeaggregatedRecords(batch.getDeaggregatedRecordSize());
+					ofNullable(batch.getMillisBehindLatest()).ifPresent(shardConsumerMetricsReporter::setMillisBehindLatest);
+				});
 
-			while (isRunning()) {
-				if (nextShardItr == null) {
+				if (result == COMPLETE) {
 					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 {
-					shardMetricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecordsPerFetch);
-					GetRecordsResult getRecordsResult = getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
-
-					List<Record> aggregatedRecords = getRecordsResult.getRecords();
-					int numberOfAggregatedRecords = aggregatedRecords.size();
-					shardMetricsReporter.setNumberOfAggregatedRecords(numberOfAggregatedRecords);
-
-					// each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
-					List<UserRecord> fetchedRecords = deaggregateRecords(
-						aggregatedRecords,
-						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-					long recordBatchSizeBytes = 0L;
-					for (UserRecord record : fetchedRecords) {
-						recordBatchSizeBytes += record.getData().remaining();
-						deserializeRecordForCollectionAndUpdateState(record);
-					}
-
-					int numberOfDeaggregatedRecords = fetchedRecords.size();
-					shardMetricsReporter.setNumberOfDeaggregatedRecords(numberOfDeaggregatedRecords);
-
-					nextShardItr = getRecordsResult.getNextShardIterator();
-
-					long adjustmentEndTimeNanos = adjustRunLoopFrequency(processingStartTimeNanos, System.nanoTime());
-					long runLoopTimeNanos = adjustmentEndTimeNanos - processingStartTimeNanos;
-					maxNumberOfRecordsPerFetch = adaptRecordsToRead(runLoopTimeNanos, fetchedRecords.size(), recordBatchSizeBytes, maxNumberOfRecordsPerFetch);
-					shardMetricsReporter.setRunLoopTimeNanos(runLoopTimeNanos);
-					processingStartTimeNanos = adjustmentEndTimeNanos; // for next time through the loop
+					startingPosition = StartingPosition.continueFromSequenceNumber(lastSequenceNum);

Review comment:
       Why does the `ShardConsumer` need to keep track and update the `startingPosition`?
   It feels like this should be an internal state to the record publisher, and also as how I understand it, the record publisher only ever respects the `nextShardItr` that it maintains.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
-
-	// AWS Kinesis has a read limit of 2 Mb/sec
-	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
-	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
-
 	private final KinesisDeserializationSchema<T> deserializer;
 
-	private final KinesisProxyInterface kinesis;
-
 	private final int subscribedShardStateIndex;
 
 	private final KinesisDataFetcher<T> fetcherRef;
 
 	private final StreamShardHandle subscribedShard;
 
-	private int maxNumberOfRecordsPerFetch;
-	private final long fetchIntervalMillis;
-	private final boolean useAdaptiveReads;
+	private final ShardConsumerMetricsReporter shardConsumerMetricsReporter;
 
-	private final ShardMetricsReporter shardMetricsReporter;
+	private StartingPosition startingPosition;
 
 	private SequenceNumber lastSequenceNum;
 
-	private Date initTimestamp;
+	private final RecordPublisher recordPublisher;
 
 	/**
 	 * Creates a shard consumer.
 	 *
 	 * @param fetcherRef reference to the owning fetcher
+	 * @param recordPublisher the record publisher used to read records from kinesis
 	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
 	 * @param subscribedShard the shard this consumer is subscribed to
 	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 * @param kinesis the proxy instance to interact with Kinesis
-	 * @param shardMetricsReporter the reporter to report metrics to
+	 * @param shardConsumerMetricsReporter the reporter to report metrics to
+	 * @param shardDeserializer used to deserialize incoming records
 	 */
 	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+						RecordPublisher recordPublisher,
 						Integer subscribedShardStateIndex,
 						StreamShardHandle subscribedShard,
 						SequenceNumber lastSequenceNum,
-						KinesisProxyInterface kinesis,
-						ShardMetricsReporter shardMetricsReporter,
+						ShardConsumerMetricsReporter shardConsumerMetricsReporter,
 						KinesisDeserializationSchema<T> shardDeserializer) {
 		this.fetcherRef = checkNotNull(fetcherRef);
+		this.recordPublisher = checkNotNull(recordPublisher);
 		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
 		this.subscribedShard = checkNotNull(subscribedShard);
+		this.shardConsumerMetricsReporter = checkNotNull(shardConsumerMetricsReporter);
 		this.lastSequenceNum = checkNotNull(lastSequenceNum);
 
-		this.shardMetricsReporter = checkNotNull(shardMetricsReporter);
-
 		checkArgument(
 			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
 			"Should not start a ShardConsumer if the shard has already been completely read.");
 
 		this.deserializer = shardDeserializer;
 
 		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
-		this.kinesis = kinesis;
-		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
-			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
-		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
-			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
-		this.useAdaptiveReads = Boolean.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS,
-			Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS)));
-
-		if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+
+		if (lastSequenceNum.equals(SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+			Date initTimestamp;
 			String timestamp = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP);
 
 			try {
 				String format = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT,
 					ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT);
 				SimpleDateFormat customDateFormat = new SimpleDateFormat(format);
-				this.initTimestamp = customDateFormat.parse(timestamp);
+				initTimestamp = customDateFormat.parse(timestamp);
 			} catch (IllegalArgumentException | NullPointerException exception) {
 				throw new IllegalArgumentException(exception);
 			} catch (ParseException exception) {
-				this.initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
+				initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
 			}
-		} else {
-			this.initTimestamp = null;
-		}
-	}
-
-	/**
-	 * Returns a shard iterator for the given {@link SequenceNumber}.
-	 *
-	 * @return shard iterator
-	 * @throws Exception
-	 */
-	protected String getShardIterator(SequenceNumber sequenceNumber) throws Exception {
 
-		if (isSentinelSequenceNumber(sequenceNumber)) {
-			return getShardIteratorForSentinel(sequenceNumber);
+			startingPosition = StartingPosition.fromTimestamp(initTimestamp);
 		} else {
-			// we will be starting from an actual sequence number (due to restore from failure).
-			return getShardIteratorForRealSequenceNumber(sequenceNumber);
+			startingPosition = StartingPosition.restartFromSequenceNumber(checkNotNull(lastSequenceNum));
 		}
 	}
 
-	protected String getShardIteratorForSentinel(SequenceNumber sentinelSequenceNumber) throws InterruptedException {

Review comment:
       👍 really nice to see this go away :)

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import java.util.function.Consumer;
+
+/**
+ * An adaptive record publisher to add a dynamic loop delay and batch read size for {@link PollingRecordPublisher}.
+ * Kinesis Streams have quotas on the transactions per second, and throughout. This class attempts to balance
+ * quotas and mitigate back off errors.
+ */
+@Internal
+public class AdaptivePollingRecordPublisher extends PollingRecordPublisher {

Review comment:
       👍 really like the approach you took in refactoring this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059) 
   * c67dc7faaf6c0ecb73295fc17755f5142c8fedb3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284) 
   * a62d1e322223e012b669d56168c198d324f8d83a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448",
       "triggerID" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   * a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-658613015


   @tzulitai this pull request is ready for your review, thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468867578



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
-
-	// AWS Kinesis has a read limit of 2 Mb/sec
-	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
-	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
-
 	private final KinesisDeserializationSchema<T> deserializer;
 
-	private final KinesisProxyInterface kinesis;
-
 	private final int subscribedShardStateIndex;
 
 	private final KinesisDataFetcher<T> fetcherRef;
 
 	private final StreamShardHandle subscribedShard;
 
-	private int maxNumberOfRecordsPerFetch;
-	private final long fetchIntervalMillis;
-	private final boolean useAdaptiveReads;
+	private final ShardConsumerMetricsReporter shardConsumerMetricsReporter;
 
-	private final ShardMetricsReporter shardMetricsReporter;
+	private StartingPosition startingPosition;
 
 	private SequenceNumber lastSequenceNum;
 
-	private Date initTimestamp;
+	private final RecordPublisher recordPublisher;
 
 	/**
 	 * Creates a shard consumer.
 	 *
 	 * @param fetcherRef reference to the owning fetcher
+	 * @param recordPublisher the record publisher used to read records from kinesis
 	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
 	 * @param subscribedShard the shard this consumer is subscribed to
 	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 * @param kinesis the proxy instance to interact with Kinesis
-	 * @param shardMetricsReporter the reporter to report metrics to
+	 * @param shardConsumerMetricsReporter the reporter to report metrics to
+	 * @param shardDeserializer used to deserialize incoming records
 	 */
 	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+						RecordPublisher recordPublisher,
 						Integer subscribedShardStateIndex,
 						StreamShardHandle subscribedShard,
 						SequenceNumber lastSequenceNum,
-						KinesisProxyInterface kinesis,
-						ShardMetricsReporter shardMetricsReporter,
+						ShardConsumerMetricsReporter shardConsumerMetricsReporter,
 						KinesisDeserializationSchema<T> shardDeserializer) {
 		this.fetcherRef = checkNotNull(fetcherRef);
+		this.recordPublisher = checkNotNull(recordPublisher);
 		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
 		this.subscribedShard = checkNotNull(subscribedShard);
+		this.shardConsumerMetricsReporter = checkNotNull(shardConsumerMetricsReporter);
 		this.lastSequenceNum = checkNotNull(lastSequenceNum);
 
-		this.shardMetricsReporter = checkNotNull(shardMetricsReporter);
-
 		checkArgument(
 			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
 			"Should not start a ShardConsumer if the shard has already been completely read.");
 
 		this.deserializer = shardDeserializer;
 
 		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
-		this.kinesis = kinesis;
-		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
-			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
-		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
-			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
-		this.useAdaptiveReads = Boolean.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS,
-			Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS)));
-
-		if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+
+		if (lastSequenceNum.equals(SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+			Date initTimestamp;
 			String timestamp = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP);
 
 			try {
 				String format = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT,
 					ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT);
 				SimpleDateFormat customDateFormat = new SimpleDateFormat(format);
-				this.initTimestamp = customDateFormat.parse(timestamp);
+				initTimestamp = customDateFormat.parse(timestamp);
 			} catch (IllegalArgumentException | NullPointerException exception) {
 				throw new IllegalArgumentException(exception);
 			} catch (ParseException exception) {
-				this.initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
+				initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
 			}
-		} else {
-			this.initTimestamp = null;
-		}
-	}
-
-	/**
-	 * Returns a shard iterator for the given {@link SequenceNumber}.
-	 *
-	 * @return shard iterator
-	 * @throws Exception
-	 */
-	protected String getShardIterator(SequenceNumber sequenceNumber) throws Exception {
 
-		if (isSentinelSequenceNumber(sequenceNumber)) {
-			return getShardIteratorForSentinel(sequenceNumber);
+			startingPosition = StartingPosition.fromTimestamp(initTimestamp);
 		} else {
-			// we will be starting from an actual sequence number (due to restore from failure).
-			return getShardIteratorForRealSequenceNumber(sequenceNumber);
+			startingPosition = StartingPosition.restartFromSequenceNumber(checkNotNull(lastSequenceNum));
 		}
 	}
 
-	protected String getShardIteratorForSentinel(SequenceNumber sentinelSequenceNumber) throws InterruptedException {

Review comment:
       👍 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418) 
   * c3fcba4e68103eb3a2283d943d7328bb6fe1cb06 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r454854457



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyMarker.java
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * A marker interface for generic Kinesis Proxy.
+ */
+@Internal

Review comment:
       It is simply a marker interface for generics on the `RecordPublisherFactory`, used to denote a `KinesisProxy`, be that SDK v1 or v2.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469762021



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -85,12 +88,8 @@
 
 		Preconditions.checkArgument(expiredIteratorBackoffMillis >= 0);
 		Preconditions.checkArgument(maxNumberOfRecordsPerFetch > 0);
-	}
 
-	@Override
-	public void initialize(StartingPosition startingPosition) throws InterruptedException {
-		nextStartingPosition = startingPosition;

Review comment:
       Ah yes I missed that Precondition. I will remove that check. 
   `nextStartingPosition` cannot be `final` as it is updated in `run` to recover in case of `ExpiredIteratorException`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r454854457



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyMarker.java
##########
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.proxy;
+
+import org.apache.flink.annotation.Internal;
+
+/**
+ * A marker interface for generic Kinesis Proxy.
+ */
+@Internal

Review comment:
       It is simply a marker interface for generics on the `RecordPublisherFactory`, used to denote a `KinesisProxy`, be that SDK v1 or v2. There is only one usage of the interface at the moment, but the next pull request will add the SDK v2 implementation




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469766625



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
##########
@@ -37,6 +46,41 @@
 	 * @param streamShardHandle the stream shard in which to consume from
 	 * @return the constructed {@link RecordPublisher}
 	 */
-	RecordPublisher create(Properties consumerConfig, MetricGroup metricGroup, StreamShardHandle streamShardHandle);
+	RecordPublisher create(
+			SequenceNumber sequenceNumber,
+			Properties consumerConfig,
+			MetricGroup metricGroup,
+			StreamShardHandle streamShardHandle) throws InterruptedException;
+
+	/**
+	 * Determines the starting position in which the {@link RecordPublisher} should start consuming from.
+	 *
+	 * @param sequenceNumber the sequence number to start from
+	 * @param consumerConfig the consumer properties
+	 * @return the {@link StartingPosition} in which to start consuming from
+	 */
+	default StartingPosition getStartingPosition(

Review comment:
       👍 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469131510



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * acdad515e90ed3af5a50ed9282c32c0c6a507e30 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415) 
   * f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812) Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) 
   * e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812) Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468875370



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(
+			final StreamShardHandle subscribedShard,
+			final PollingRecordPublisherMetricsReporter metricsReporter,
+			final KinesisProxyInterface kinesisProxy,
+			final int maxNumberOfRecordsPerFetch,
+			final long expiredIteratorBackoffMillis) {
+		this.subscribedShard = subscribedShard;
+		this.metricsReporter = metricsReporter;
+		this.kinesisProxy = kinesisProxy;
+		this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+		this.expiredIteratorBackoffMillis = expiredIteratorBackoffMillis;
+	}
+
+	@Override
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer) throws InterruptedException {
+		return run(startingPosition, consumer, maxNumberOfRecordsPerFetch);
+	}
+
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer, int maxNumberOfRecords) throws InterruptedException {
+		if (nextShardItr == null) {
+			nextShardItr = getShardIterator(startingPosition);
+		}
+
+		if (nextShardItr == null) {
+			return COMPLETE;
+		}
+
+		metricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecords);
+
+		GetRecordsResult result = getRecords(nextShardItr, startingPosition, maxNumberOfRecords);
+
+		consumer.accept(new RecordBatch(result.getRecords(), subscribedShard, result.getMillisBehindLatest()));
+
+		nextShardItr = result.getNextShardIterator();
+		return nextShardItr == null ? COMPLETE : INCOMPLETE;
+	}
+
+	/**
+	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
+	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
+	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
+	 * be used for the next call to this method.
+	 *
+	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
+	 * fully collected with {@code ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
+	 * {@code ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
+	 * incorrect shard iteration if the iterator had to be refreshed.
+	 *
+	 * @param shardItr shard iterator to use
+	 * @param startingPosition the position in the stream in which to consume from
+	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
+	 * @return get records result
+	 */
+	private GetRecordsResult getRecords(String shardItr, final StartingPosition startingPosition, int maxNumberOfRecords) throws InterruptedException {
+		GetRecordsResult getRecordsResult = null;
+		while (getRecordsResult == null) {
+			try {
+				getRecordsResult = kinesisProxy.getRecords(shardItr, maxNumberOfRecords);
+			} catch (ExpiredIteratorException eiEx) {
+				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
+					" refreshing the iterator ...", shardItr, subscribedShard);
+
+				shardItr = getShardIterator(startingPosition);
+
+				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
+				if (expiredIteratorBackoffMillis != 0) {
+					Thread.sleep(expiredIteratorBackoffMillis);
+				}
+			}
+		}
+		return getRecordsResult;
+	}
+
+	/**
+	 * Returns a shard iterator for the given {@link SequenceNumber}.
+	 *
+	 * @return shard iterator
+	 */
+	@Nullable
+	private String getShardIterator(final StartingPosition startingPosition) throws InterruptedException {
+		if (startingPosition.getShardIteratorType() == LATEST && subscribedShard.isClosed()) {
+			return null;
+		}
+
+		return kinesisProxy.getShardIterator(

Review comment:
       For timestamp the AWS client throws an exception if `time > now`. If `time <= now` it returns `null` if the shard is complete, otherwise the iterator.
   
   If a sequence number is non-existent then the service throws a 400 client error and the application terminates. If the sequence number is invalid then the AWS client throws an exception.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448",
       "triggerID" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ea015e685d16e8fe865aed212c3fda70a80edc0",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5493",
       "triggerID" : "1ea015e685d16e8fe865aed212c3fda70a80edc0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   * 1ea015e685d16e8fe865aed212c3fda70a80edc0 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5493) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448",
       "triggerID" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c3fcba4e68103eb3a2283d943d7328bb6fe1cb06 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445) 
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   * a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-672785130


   @tzulitai I have addressed all the comments and updated the EFO PR to the new `RecordPublisher` interface. Thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c3fcba4e68103eb3a2283d943d7328bb6fe1cb06 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445) 
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   * a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tzulitai commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
tzulitai commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469749157



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
##########
@@ -37,6 +46,41 @@
 	 * @param streamShardHandle the stream shard in which to consume from
 	 * @return the constructed {@link RecordPublisher}
 	 */
-	RecordPublisher create(Properties consumerConfig, MetricGroup metricGroup, StreamShardHandle streamShardHandle);
+	RecordPublisher create(
+			SequenceNumber sequenceNumber,
+			Properties consumerConfig,
+			MetricGroup metricGroup,
+			StreamShardHandle streamShardHandle) throws InterruptedException;
+
+	/**
+	 * Determines the starting position in which the {@link RecordPublisher} should start consuming from.
+	 *
+	 * @param sequenceNumber the sequence number to start from
+	 * @param consumerConfig the consumer properties
+	 * @return the {@link StartingPosition} in which to start consuming from
+	 */
+	default StartingPosition getStartingPosition(

Review comment:
       Was there any particular reason to put this method in the `RecordPublisherFactory`, instead of letting the factory get a `StartingPosition` (and not a `SequenceNumber`) on `create`?
   The conversion from sequence number to `StartingPosition` logic could probably be a separate static utility method somewhere that is unrelated to the actual implementation of the `RecordPublisherFactory`.

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -85,12 +88,8 @@
 
 		Preconditions.checkArgument(expiredIteratorBackoffMillis >= 0);
 		Preconditions.checkArgument(maxNumberOfRecordsPerFetch > 0);
-	}
 
-	@Override
-	public void initialize(StartingPosition startingPosition) throws InterruptedException {
-		nextStartingPosition = startingPosition;

Review comment:
       With the `intialize` method now removed, could you also remove the preconditions check on line 101?
   `nextStartingPosition` should now never be null.
   Also, can `nextStartingPosition` now be a final field?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433) 
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c67dc7faaf6c0ecb73295fc17755f5142c8fedb3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284) 
   * a62d1e322223e012b669d56168c198d324f8d83a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657428114


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8 (Mon Jul 13 09:08:13 UTC 2020)
   
   **Warnings:**
    * **1 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468863930



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisher.java
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+
+import java.util.function.Consumer;
+
+/**
+ * A {@code RecordPublisher} will consume records from an external stream and deliver them to the registered subscriber.
+ */
+@Internal
+public interface RecordPublisher {
+
+	/**
+	 * Run the record publisher. Records will be consumed from the stream and published to the consumer.
+	 * The number of batches retrieved by a single invocation will vary based on implementation.
+	 *
+	 * @param startingPosition the position in the stream from which to consume
+	 * @param recordConsumer the record consumer in which to output records
+	 * @return a status enum to represent whether a shard has been fully consumed
+	 * @throws InterruptedException
+	 */
+	RecordPublisherRunResult run(StartingPosition startingPosition, Consumer<RecordBatch> recordConsumer) throws InterruptedException;

Review comment:
       I need to double check this, I will follow up tomorrow. Essentially it was originally like this for EFO. As the EFO subscription passes records to the consumer in a callback it is still in the network thread. This means `ReadTimeout` can occur and the consumer could fail mid batch (If you apply very large backpressure). The `FanOutRecordPublisher` would then not know where to start consumption from, therefore the `ShardConsumer` was passing that in. However, I ended up splitting the network and shard consumer thread with a `BlockingQueue` in the end, so I can probably track the state in the `RecordPublisher` as you suggest.
   
   I will investigate and get back to you.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] asfgit closed pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #12881:
URL: https://github.com/apache/flink/pull/12881


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b2f00606dff3ce4dfda09054cb2d89b317788b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016) 
   * 44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448",
       "triggerID" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ea015e685d16e8fe865aed212c3fda70a80edc0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1ea015e685d16e8fe865aed212c3fda70a80edc0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   * a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448) 
   * 1ea015e685d16e8fe865aed212c3fda70a80edc0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468857715



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 2b2f00606dff3ce4dfda09054cb2d89b317788b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016) 
   * 44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433) 
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tzulitai commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
tzulitai commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-675360727


   LGTM, +1.
   Thanks @dannycranmer, will merge this PR now after another CI run.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418) 
   * c3fcba4e68103eb3a2283d943d7328bb6fe1cb06 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r454856500



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(
+			final StreamShardHandle subscribedShard,
+			final PollingRecordPublisherMetricsReporter metricsReporter,
+			final KinesisProxyInterface kinesisProxy,
+			final int maxNumberOfRecordsPerFetch,
+			final long expiredIteratorBackoffMillis) {
+		this.subscribedShard = subscribedShard;
+		this.metricsReporter = metricsReporter;
+		this.kinesisProxy = kinesisProxy;
+		this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch;
+		this.expiredIteratorBackoffMillis = expiredIteratorBackoffMillis;
+	}
+
+	@Override
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer) throws InterruptedException {
+		return run(startingPosition, consumer, maxNumberOfRecordsPerFetch);
+	}
+
+	public RecordPublisherRunResult run(final StartingPosition startingPosition, final Consumer<RecordBatch> consumer, int maxNumberOfRecords) throws InterruptedException {
+		if (nextShardItr == null) {
+			nextShardItr = getShardIterator(startingPosition);
+		}
+
+		if (nextShardItr == null) {
+			return COMPLETE;
+		}
+
+		metricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecords);
+
+		GetRecordsResult result = getRecords(nextShardItr, startingPosition, maxNumberOfRecords);
+
+		consumer.accept(new RecordBatch(result.getRecords(), subscribedShard, result.getMillisBehindLatest()));
+
+		nextShardItr = result.getNextShardIterator();
+		return nextShardItr == null ? COMPLETE : INCOMPLETE;
+	}
+
+	/**
+	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
+	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
+	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
+	 * be used for the next call to this method.
+	 *
+	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
+	 * fully collected with {@code ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
+	 * {@code ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
+	 * incorrect shard iteration if the iterator had to be refreshed.
+	 *
+	 * @param shardItr shard iterator to use
+	 * @param startingPosition the position in the stream in which to consume from
+	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
+	 * @return get records result
+	 */
+	private GetRecordsResult getRecords(String shardItr, final StartingPosition startingPosition, int maxNumberOfRecords) throws InterruptedException {
+		GetRecordsResult getRecordsResult = null;
+		while (getRecordsResult == null) {
+			try {
+				getRecordsResult = kinesisProxy.getRecords(shardItr, maxNumberOfRecords);
+			} catch (ExpiredIteratorException | InterruptedException eiEx) {

Review comment:
       Ah that `InterruptedException` should not be there! I will delete that
   
   The other exceptions are handled in the proxy: https://github.com/apache/flink/blob/c5915cf87f96e1c7ebd84ad00f7eabade7e7fe37/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java#L248




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468875478



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordBatch.java
##########
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+
+import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord;
+import com.amazonaws.services.kinesis.model.Record;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.List;
+
+/**
+ * A batch of UserRecords received from Kinesis.
+ * Input records are de-aggregated using KCL 1.x library.
+ * It is expected that AWS SDK v2.x messages are converted to KCL 1.x {@link UserRecord}.
+ */
+public class RecordBatch {
+
+	private final int aggregatedRecordSize;
+
+	private final List<UserRecord> deaggregatedRecords;
+
+	private final long totalSizeInBytes;
+
+	private final Long millisBehindLatest;
+
+	public RecordBatch(final List<Record> records, final StreamShardHandle subscribedShard, final Long millisBehindLatest) {
+		this.aggregatedRecordSize = records.size();

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-672279027


   @tzulitai I have addressed a few of the comments, I will answer and address the rest tomorrow. Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812) Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) 
   * e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469125201



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -53,283 +49,111 @@
  */
 @Internal
 public class ShardConsumer<T> implements Runnable {
-
-	private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class);
-
-	// AWS Kinesis has a read limit of 2 Mb/sec
-	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
-	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
-
 	private final KinesisDeserializationSchema<T> deserializer;
 
-	private final KinesisProxyInterface kinesis;
-
 	private final int subscribedShardStateIndex;
 
 	private final KinesisDataFetcher<T> fetcherRef;
 
 	private final StreamShardHandle subscribedShard;
 
-	private int maxNumberOfRecordsPerFetch;
-	private final long fetchIntervalMillis;
-	private final boolean useAdaptiveReads;
+	private final ShardConsumerMetricsReporter shardConsumerMetricsReporter;
 
-	private final ShardMetricsReporter shardMetricsReporter;
+	private StartingPosition startingPosition;
 
 	private SequenceNumber lastSequenceNum;
 
-	private Date initTimestamp;
+	private final RecordPublisher recordPublisher;
 
 	/**
 	 * Creates a shard consumer.
 	 *
 	 * @param fetcherRef reference to the owning fetcher
+	 * @param recordPublisher the record publisher used to read records from kinesis
 	 * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to
 	 * @param subscribedShard the shard this consumer is subscribed to
 	 * @param lastSequenceNum the sequence number in the shard to start consuming
-	 * @param kinesis the proxy instance to interact with Kinesis
-	 * @param shardMetricsReporter the reporter to report metrics to
+	 * @param shardConsumerMetricsReporter the reporter to report metrics to
+	 * @param shardDeserializer used to deserialize incoming records
 	 */
 	public ShardConsumer(KinesisDataFetcher<T> fetcherRef,
+						RecordPublisher recordPublisher,
 						Integer subscribedShardStateIndex,
 						StreamShardHandle subscribedShard,
 						SequenceNumber lastSequenceNum,
-						KinesisProxyInterface kinesis,
-						ShardMetricsReporter shardMetricsReporter,
+						ShardConsumerMetricsReporter shardConsumerMetricsReporter,
 						KinesisDeserializationSchema<T> shardDeserializer) {
 		this.fetcherRef = checkNotNull(fetcherRef);
+		this.recordPublisher = checkNotNull(recordPublisher);
 		this.subscribedShardStateIndex = checkNotNull(subscribedShardStateIndex);
 		this.subscribedShard = checkNotNull(subscribedShard);
+		this.shardConsumerMetricsReporter = checkNotNull(shardConsumerMetricsReporter);
 		this.lastSequenceNum = checkNotNull(lastSequenceNum);
 
-		this.shardMetricsReporter = checkNotNull(shardMetricsReporter);
-
 		checkArgument(
 			!lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get()),
 			"Should not start a ShardConsumer if the shard has already been completely read.");
 
 		this.deserializer = shardDeserializer;
 
 		Properties consumerConfig = fetcherRef.getConsumerConfiguration();
-		this.kinesis = kinesis;
-		this.maxNumberOfRecordsPerFetch = Integer.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_MAX,
-			Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX)));
-		this.fetchIntervalMillis = Long.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS,
-			Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS)));
-		this.useAdaptiveReads = Boolean.valueOf(consumerConfig.getProperty(
-			ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS,
-			Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS)));
-
-		if (lastSequenceNum.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+
+		if (lastSequenceNum.equals(SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
+			Date initTimestamp;
 			String timestamp = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP);
 
 			try {
 				String format = consumerConfig.getProperty(ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT,
 					ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT);
 				SimpleDateFormat customDateFormat = new SimpleDateFormat(format);
-				this.initTimestamp = customDateFormat.parse(timestamp);
+				initTimestamp = customDateFormat.parse(timestamp);
 			} catch (IllegalArgumentException | NullPointerException exception) {
 				throw new IllegalArgumentException(exception);
 			} catch (ParseException exception) {
-				this.initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
+				initTimestamp = new Date((long) (Double.parseDouble(timestamp) * 1000));
 			}
-		} else {
-			this.initTimestamp = null;
-		}
-	}
-
-	/**
-	 * Returns a shard iterator for the given {@link SequenceNumber}.
-	 *
-	 * @return shard iterator
-	 * @throws Exception
-	 */
-	protected String getShardIterator(SequenceNumber sequenceNumber) throws Exception {
 
-		if (isSentinelSequenceNumber(sequenceNumber)) {
-			return getShardIteratorForSentinel(sequenceNumber);
+			startingPosition = StartingPosition.fromTimestamp(initTimestamp);
 		} else {
-			// we will be starting from an actual sequence number (due to restore from failure).
-			return getShardIteratorForRealSequenceNumber(sequenceNumber);
+			startingPosition = StartingPosition.restartFromSequenceNumber(checkNotNull(lastSequenceNum));
 		}
 	}
 
-	protected String getShardIteratorForSentinel(SequenceNumber sentinelSequenceNumber) throws InterruptedException {
-		String nextShardItr;
-
-		if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get())) {
-			// if the shard is already closed, there will be no latest next record to get for this shard
-			if (subscribedShard.isClosed()) {
-				nextShardItr = null;
-			} else {
-				nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.LATEST.toString(), null);
-			}
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get())) {
-			nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.TRIM_HORIZON.toString(), null);
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get())) {
-			nextShardItr = null;
-		} else if (sentinelSequenceNumber.equals(SentinelSequenceNumber.SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM.get())) {
-			nextShardItr = kinesis.getShardIterator(subscribedShard, ShardIteratorType.AT_TIMESTAMP.toString(), initTimestamp);
-		} else {
-			throw new RuntimeException("Unknown sentinel type: " + sentinelSequenceNumber);
-		}
-
-		return nextShardItr;
-	}
-
-	protected String getShardIteratorForRealSequenceNumber(SequenceNumber sequenceNumber)
-			throws Exception {
-
-		// if the last sequence number refers to an aggregated record, we need to clean up any dangling sub-records
-		// from the last aggregated record; otherwise, we can simply start iterating from the record right after.
-
-		if (sequenceNumber.isAggregated()) {
-			return getShardIteratorForAggregatedSequenceNumber(sequenceNumber);
-		} else {
-			// the last record was non-aggregated, so we can simply start from the next record
-			return kinesis.getShardIterator(
-					subscribedShard,
-					ShardIteratorType.AFTER_SEQUENCE_NUMBER.toString(),
-					sequenceNumber.getSequenceNumber());
-		}
-	}
-
-	protected String getShardIteratorForAggregatedSequenceNumber(SequenceNumber sequenceNumber)
-			throws Exception {
-
-		String itrForLastAggregatedRecord =
-				kinesis.getShardIterator(
-						subscribedShard,
-						ShardIteratorType.AT_SEQUENCE_NUMBER.toString(),
-						sequenceNumber.getSequenceNumber());
-
-		// get only the last aggregated record
-		GetRecordsResult getRecordsResult = getRecords(itrForLastAggregatedRecord, 1);
-
-		List<UserRecord> fetchedRecords = deaggregateRecords(
-				getRecordsResult.getRecords(),
-				subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-				subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-		long lastSubSequenceNum = sequenceNumber.getSubSequenceNumber();
-		for (UserRecord record : fetchedRecords) {
-			// we have found a dangling sub-record if it has a larger subsequence number
-			// than our last sequence number; if so, collect the record and update state
-			if (record.getSubSequenceNumber() > lastSubSequenceNum) {
-				deserializeRecordForCollectionAndUpdateState(record);
-			}
-		}
-
-		return getRecordsResult.getNextShardIterator();
-	}
-
-	@SuppressWarnings("unchecked")
 	@Override
 	public void run() {
 		try {
-			String nextShardItr = getShardIterator(lastSequenceNum);
+			while (isRunning()) {
+				final RecordPublisherRunResult result = recordPublisher.run(startingPosition, batch -> {
+					for (UserRecord userRecord : batch.getDeaggregatedRecords()) {
+						if (filterDeaggregatedRecord(userRecord)) {
+							deserializeRecordForCollectionAndUpdateState(userRecord);
+						}
+					}
 
-			long processingStartTimeNanos = System.nanoTime();
+					shardConsumerMetricsReporter.setAverageRecordSizeBytes(batch.getAverageRecordSizeBytes());
+					shardConsumerMetricsReporter.setNumberOfAggregatedRecords(batch.getAggregatedRecordSize());
+					shardConsumerMetricsReporter.setNumberOfDeaggregatedRecords(batch.getDeaggregatedRecordSize());
+					ofNullable(batch.getMillisBehindLatest()).ifPresent(shardConsumerMetricsReporter::setMillisBehindLatest);
+				});
 
-			while (isRunning()) {
-				if (nextShardItr == null) {
+				if (result == COMPLETE) {
 					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 {
-					shardMetricsReporter.setMaxNumberOfRecordsPerFetch(maxNumberOfRecordsPerFetch);
-					GetRecordsResult getRecordsResult = getRecords(nextShardItr, maxNumberOfRecordsPerFetch);
-
-					List<Record> aggregatedRecords = getRecordsResult.getRecords();
-					int numberOfAggregatedRecords = aggregatedRecords.size();
-					shardMetricsReporter.setNumberOfAggregatedRecords(numberOfAggregatedRecords);
-
-					// each of the Kinesis records may be aggregated, so we must deaggregate them before proceeding
-					List<UserRecord> fetchedRecords = deaggregateRecords(
-						aggregatedRecords,
-						subscribedShard.getShard().getHashKeyRange().getStartingHashKey(),
-						subscribedShard.getShard().getHashKeyRange().getEndingHashKey());
-
-					long recordBatchSizeBytes = 0L;
-					for (UserRecord record : fetchedRecords) {
-						recordBatchSizeBytes += record.getData().remaining();
-						deserializeRecordForCollectionAndUpdateState(record);
-					}
-
-					int numberOfDeaggregatedRecords = fetchedRecords.size();
-					shardMetricsReporter.setNumberOfDeaggregatedRecords(numberOfDeaggregatedRecords);
-
-					nextShardItr = getRecordsResult.getNextShardIterator();
-
-					long adjustmentEndTimeNanos = adjustRunLoopFrequency(processingStartTimeNanos, System.nanoTime());
-					long runLoopTimeNanos = adjustmentEndTimeNanos - processingStartTimeNanos;
-					maxNumberOfRecordsPerFetch = adaptRecordsToRead(runLoopTimeNanos, fetchedRecords.size(), recordBatchSizeBytes, maxNumberOfRecordsPerFetch);
-					shardMetricsReporter.setRunLoopTimeNanos(runLoopTimeNanos);
-					processingStartTimeNanos = adjustmentEndTimeNanos; // for next time through the loop
+					startingPosition = StartingPosition.continueFromSequenceNumber(lastSequenceNum);

Review comment:
       It does not anymore.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469127962



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisher.java
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+
+import java.util.function.Consumer;
+
+/**
+ * A {@code RecordPublisher} will consume records from an external stream and deliver them to the registered subscriber.
+ */
+@Internal
+public interface RecordPublisher {
+
+	/**
+	 * Run the record publisher. Records will be consumed from the stream and published to the consumer.
+	 * The number of batches retrieved by a single invocation will vary based on implementation.
+	 *
+	 * @param startingPosition the position in the stream from which to consume
+	 * @param recordConsumer the record consumer in which to output records
+	 * @return a status enum to represent whether a shard has been fully consumed
+	 * @throws InterruptedException
+	 */
+	RecordPublisherRunResult run(StartingPosition startingPosition, Consumer<RecordBatch> recordConsumer) throws InterruptedException;

Review comment:
       @tzulitai I have refactored the code, turned out to be a substantial change to move 1 field!
   - `RecordPublisher` now tracks position in stream, `StartingPosition` is only used at startup passed by the factory
   - `ShardConsumer` now passes back the last consumed sequence number in the callback. This is required to allow the `PollingRecordPublisher` to recover in event of `ExpiredIterator`
   - `ShardConsumer` still needs to track the last consumed sequence number for deaggregate filtering
   
   Let me know what you think. 👍 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a62d1e322223e012b669d56168c198d324f8d83a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287) 
   * acdad515e90ed3af5a50ed9282c32c0c6a507e30 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468859151



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import java.util.function.Consumer;
+
+/**
+ * An adaptive record publisher to add a dynamic loop delay and batch read size for {@link PollingRecordPublisher}.
+ * Kinesis Streams have quotas on the transactions per second, and throughout. This class attempts to balance
+ * quotas and mitigate back off errors.
+ */
+@Internal
+public class AdaptivePollingRecordPublisher extends PollingRecordPublisher {

Review comment:
       Thanks! 😄 

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.model;
+
+import com.amazonaws.services.kinesis.model.ShardIteratorType;
+
+import javax.annotation.Nullable;
+
+import java.util.Date;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.TRIM_HORIZON;
+import static org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber.isSentinelSequenceNumber;
+
+/**
+ * The position in which to start consuming from a stream.
+ */
+public class StartingPosition {

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469763944



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
##########
@@ -37,6 +46,41 @@
 	 * @param streamShardHandle the stream shard in which to consume from
 	 * @return the constructed {@link RecordPublisher}
 	 */
-	RecordPublisher create(Properties consumerConfig, MetricGroup metricGroup, StreamShardHandle streamShardHandle);
+	RecordPublisher create(
+			SequenceNumber sequenceNumber,
+			Properties consumerConfig,
+			MetricGroup metricGroup,
+			StreamShardHandle streamShardHandle) throws InterruptedException;
+
+	/**
+	 * Determines the starting position in which the {@link RecordPublisher} should start consuming from.
+	 *
+	 * @param sequenceNumber the sequence number to start from
+	 * @param consumerConfig the consumer properties
+	 * @return the {@link StartingPosition} in which to start consuming from
+	 */
+	default StartingPosition getStartingPosition(

Review comment:
       No reason in particular, I am happy to move it. I am thinking a good place could be in `StartingPosition` itself, as we already have `StartingPosition.restartFromSequenceNumber(SequenceNumber)`. I could add  `StartingPosition.restartFromSequenceNumber(SequenceNumber, Properties)`, although it does feel like that class should not know how to parse properties, it might be better than creating a new class for that single method. 
   
   I will go with this and can change it if you disagree.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468857567



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/AdaptivePollingRecordPublisher.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import java.util.function.Consumer;
+
+/**
+ * An adaptive record publisher to add a dynamic loop delay and batch read size for {@link PollingRecordPublisher}.
+ * Kinesis Streams have quotas on the transactions per second, and throughout. This class attempts to balance
+ * quotas and mitigate back off errors.
+ */
+@Internal
+public class AdaptivePollingRecordPublisher extends PollingRecordPublisher {
+	// AWS Kinesis has a read limit of 2 Mb/sec
+	// https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html
+	private static final long KINESIS_SHARD_BYTES_PER_SECOND_LIMIT = 2 * 1024L * 1024L;
+
+	private int lastRecordBatchSize = 0;
+
+	private long lastRecordBatchSizeInBytes = 0;
+
+	private long processingStartTimeNanos = System.nanoTime();
+
+	private int maxNumberOfRecordsPerFetch;
+
+	private final long fetchIntervalMillis;
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	public AdaptivePollingRecordPublisher(

Review comment:
       Done

##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/polling/PollingRecordPublisher.java
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.internals.publisher.polling;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordBatch;
+import org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher;
+import org.apache.flink.streaming.connectors.kinesis.metrics.PollingRecordPublisherMetricsReporter;
+import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
+import org.apache.flink.streaming.connectors.kinesis.model.StartingPosition;
+import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle;
+import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
+
+import com.amazonaws.services.kinesis.model.ExpiredIteratorException;
+import com.amazonaws.services.kinesis.model.GetRecordsResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.function.Consumer;
+
+import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.COMPLETE;
+import static org.apache.flink.streaming.connectors.kinesis.internals.publisher.RecordPublisher.RecordPublisherRunResult.INCOMPLETE;
+
+/**
+ * A {@link RecordPublisher} that will read records from Kinesis and forward them to the subscriber.
+ * Records are consumed by polling the GetRecords KDS API using a ShardIterator.
+ */
+@Internal
+public class PollingRecordPublisher implements RecordPublisher {
+
+	private static final Logger LOG = LoggerFactory.getLogger(PollingRecordPublisher.class);
+
+	private final PollingRecordPublisherMetricsReporter metricsReporter;
+
+	private final KinesisProxyInterface kinesisProxy;
+
+	private final StreamShardHandle subscribedShard;
+
+	private String nextShardItr;
+
+	private final int maxNumberOfRecordsPerFetch;
+
+	private final long expiredIteratorBackoffMillis;
+
+	/**
+	 * A Polling implementation of {@link RecordPublisher} that polls kinesis for records.
+	 * The following KDS services are used: GetRecords and GetShardIterator.
+	 *
+	 * @param subscribedShard the shard in which to consume from
+	 * @param metricsReporter a metric reporter used to output metrics
+	 * @param kinesisProxy the proxy used to communicate with kinesis
+	 * @param maxNumberOfRecordsPerFetch the maximum number of records to retrieve per batch
+	 * @param expiredIteratorBackoffMillis the duration to sleep in the event of an {@link ExpiredIteratorException}
+	 */
+	public PollingRecordPublisher(
+			final StreamShardHandle subscribedShard,
+			final PollingRecordPublisherMetricsReporter metricsReporter,
+			final KinesisProxyInterface kinesisProxy,
+			final int maxNumberOfRecordsPerFetch,
+			final long expiredIteratorBackoffMillis) {
+		this.subscribedShard = subscribedShard;

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a62d1e322223e012b669d56168c198d324f8d83a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287) 
   * acdad515e90ed3af5a50ed9282c32c0c6a507e30 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415) 
   * f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r468859029



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -379,49 +206,18 @@ private void deserializeRecordForCollectionAndUpdateState(UserRecord record)
 	}
 
 	/**
-	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
-	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
-	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
-	 * be used for the next call to this method.
+	 * Filters out aggregated records that have previously been processed.
+	 * This method is to support restarting from a partially consumed aggregated sequence number.
 	 *
-	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
-	 * fully collected with {@link ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
-	 * {@link ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
-	 * incorrect shard iteration if the iterator had to be refreshed.
-	 *
-	 * @param shardItr shard iterator to use
-	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
-	 * @return get records result
-	 * @throws InterruptedException
+	 * @param record the record to filter
+	 * @return {@code true} if the record should be retained
 	 */
-	private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) throws Exception {
-		GetRecordsResult getRecordsResult = null;
-		while (getRecordsResult == null) {
-			try {
-				getRecordsResult = kinesis.getRecords(shardItr, maxNumberOfRecords);
-
-				// Update millis behind latest so it gets reported by the millisBehindLatest gauge
-				Long millisBehindLatest = getRecordsResult.getMillisBehindLatest();
-				if (millisBehindLatest != null) {
-					shardMetricsReporter.setMillisBehindLatest(millisBehindLatest);
-				}
-			} catch (ExpiredIteratorException eiEx) {
-				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
-					" refreshing the iterator ...", shardItr, subscribedShard);
-
-				shardItr = getShardIterator(lastSequenceNum);
-
-				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
-				if (fetchIntervalMillis != 0) {
-					Thread.sleep(fetchIntervalMillis);
-				}
-			}
+	private boolean filterDeaggregatedRecord(final UserRecord record) {
+		if (lastSequenceNum.isAggregated()) {
+			return !record.getSequenceNumber().equals(lastSequenceNum.getSequenceNumber()) ||
+				record.getSubSequenceNumber() > lastSequenceNum.getSubSequenceNumber();
 		}
-		return getRecordsResult;
-	}
 
-	@SuppressWarnings("unchecked")
-	protected static List<UserRecord> deaggregateRecords(List<Record> records, String startingHashKey, String endingHashKey) {
-		return UserRecord.deaggregate(records, new BigInteger(startingHashKey), new BigInteger(endingHashKey));
+		return true;

Review comment:
       Done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * b8e47e8baef39de33c2336533426b4fa6d07dc66 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527) Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c67dc7faaf6c0ecb73295fc17755f5142c8fedb3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284) 
   * a62d1e322223e012b669d56168c198d324f8d83a Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059) 
   * c67dc7faaf6c0ecb73295fc17755f5142c8fedb3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469796613



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
##########
@@ -37,6 +46,41 @@
 	 * @param streamShardHandle the stream shard in which to consume from
 	 * @return the constructed {@link RecordPublisher}
 	 */
-	RecordPublisher create(Properties consumerConfig, MetricGroup metricGroup, StreamShardHandle streamShardHandle);
+	RecordPublisher create(
+			SequenceNumber sequenceNumber,
+			Properties consumerConfig,
+			MetricGroup metricGroup,
+			StreamShardHandle streamShardHandle) throws InterruptedException;
+
+	/**
+	 * Determines the starting position in which the {@link RecordPublisher} should start consuming from.
+	 *
+	 * @param sequenceNumber the sequence number to start from
+	 * @param consumerConfig the consumer properties
+	 * @return the {@link StartingPosition} in which to start consuming from
+	 */
+	default StartingPosition getStartingPosition(

Review comment:
       It didn't feel right to pass the `@Nullable` fields to `StartingPosition` so in the end:
   - `Date` is parsed from properties in `KinesisConfigUtil`
   - Logic to determine to restart from timestamp or sequence is in `AWSUtil`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a62d1e322223e012b669d56168c198d324f8d83a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287) 
   * acdad515e90ed3af5a50ed9282c32c0c6a507e30 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059) 
   * c67dc7faaf6c0ecb73295fc17755f5142c8fedb3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] xiaolong-sn commented on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
xiaolong-sn commented on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-658068518


   @flinkbot run azure


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * acdad515e90ed3af5a50ed9282c32c0c6a507e30 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415) 
   * f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] tzulitai commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
tzulitai commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r469765875



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/RecordPublisherFactory.java
##########
@@ -37,6 +46,41 @@
 	 * @param streamShardHandle the stream shard in which to consume from
 	 * @return the constructed {@link RecordPublisher}
 	 */
-	RecordPublisher create(Properties consumerConfig, MetricGroup metricGroup, StreamShardHandle streamShardHandle);
+	RecordPublisher create(
+			SequenceNumber sequenceNumber,
+			Properties consumerConfig,
+			MetricGroup metricGroup,
+			StreamShardHandle streamShardHandle) throws InterruptedException;
+
+	/**
+	 * Determines the starting position in which the {@link RecordPublisher} should start consuming from.
+	 *
+	 * @param sequenceNumber the sequence number to start from
+	 * @param consumerConfig the consumer properties
+	 * @return the {@link StartingPosition} in which to start consuming from
+	 */
+	default StartingPosition getStartingPosition(

Review comment:
       That sounds good to me, thanks.
   If we want to be extra explicit, perhaps only the required properties are parsed and then passed in to `StartingPosition.restartFromSequenceNumber`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r454860445



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/StartingPosition.java
##########
@@ -0,0 +1,106 @@
+/*

Review comment:
       We cannot use `SentinelSequenceNumber` to achieve what this class does, as `SentinelSequenceNumber` is an enum. We could get rid of `SentinelSequenceNumber` in favour of this, however that class has a lot of references and would be a big refactor. Also I think semantically they differ.
   
   Also looking at the code, `SentinelSequenceNumber` is potentially serialised and therefore we would need to migrate snapshots or handle deserialisation mappings which is risky. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448",
       "triggerID" : "a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1ea015e685d16e8fe865aed212c3fda70a80edc0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5493",
       "triggerID" : "1ea015e685d16e8fe865aed212c3fda70a80edc0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   * a4a5a4e9d00bd1966f5d1a3bef551a31ca830ef1 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5448) 
   * 1ea015e685d16e8fe865aed212c3fda70a80edc0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5493) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a62d1e322223e012b669d56168c198d324f8d83a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12881:
URL: https://github.com/apache/flink/pull/12881#issuecomment-657437295


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4433",
       "triggerID" : "83c9aafd6d84a13d403d3bca66f6d6914ac6ddb8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4812",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "b8e47e8baef39de33c2336533426b4fa6d07dc66",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4527",
       "triggerID" : "658068518",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4824",
       "triggerID" : "e6ba6fc3406a7d41ea849a8fefd91a6f2bc70bee",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=4935",
       "triggerID" : "cfbeb5bc957d19c73a736a1c1507f3c070d298da",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5016",
       "triggerID" : "2b2f00606dff3ce4dfda09054cb2d89b317788b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5059",
       "triggerID" : "44aa907f7a0d7457bfd26d5294a0d77ddb4e3c09",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5284",
       "triggerID" : "c67dc7faaf6c0ecb73295fc17755f5142c8fedb3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5287",
       "triggerID" : "a62d1e322223e012b669d56168c198d324f8d83a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5415",
       "triggerID" : "acdad515e90ed3af5a50ed9282c32c0c6a507e30",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5418",
       "triggerID" : "f8f6ac06bd414eaaadb2e0bfa3f422b1925f7dfc",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445",
       "triggerID" : "c3fcba4e68103eb3a2283d943d7328bb6fe1cb06",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c3fcba4e68103eb3a2283d943d7328bb6fe1cb06 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=5445) 
   * 0e85ff32e02e3cf6432bc5a2bdcb5ca504f390a7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] dannycranmer commented on a change in pull request #12881: [FLINK-18512][kinesis] Introducing RecordPublisher. Refactor ShardConsumer to use PollingRecordPublisher

Posted by GitBox <gi...@apache.org>.
dannycranmer commented on a change in pull request #12881:
URL: https://github.com/apache/flink/pull/12881#discussion_r454853969



##########
File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java
##########
@@ -379,49 +205,18 @@ private void deserializeRecordForCollectionAndUpdateState(UserRecord record)
 	}
 
 	/**
-	 * Calls {@link KinesisProxyInterface#getRecords(String, int)}, while also handling unexpected
-	 * AWS {@link ExpiredIteratorException}s to assure that we get results and don't just fail on
-	 * such occasions. The returned shard iterator within the successful {@link GetRecordsResult} should
-	 * be used for the next call to this method.
-	 *
-	 * <p>Note: it is important that this method is not called again before all the records from the last result have been
-	 * fully collected with {@link ShardConsumer#deserializeRecordForCollectionAndUpdateState(UserRecord)}, otherwise
-	 * {@link ShardConsumer#lastSequenceNum} may refer to a sub-record in the middle of an aggregated record, leading to
-	 * incorrect shard iteration if the iterator had to be refreshed.
+	 * Filters out aggregated records that have previously been processed.
+	 * This method is to support restarting from a partially consumed aggregated sequence number.
 	 *
-	 * @param shardItr shard iterator to use
-	 * @param maxNumberOfRecords the maximum number of records to fetch for this getRecords attempt
-	 * @return get records result
-	 * @throws InterruptedException
+	 * @param record the record to filter
+	 * @return {@code true} if the record should be retained
 	 */
-	private GetRecordsResult getRecords(String shardItr, int maxNumberOfRecords) throws Exception {
-		GetRecordsResult getRecordsResult = null;
-		while (getRecordsResult == null) {
-			try {
-				getRecordsResult = kinesis.getRecords(shardItr, maxNumberOfRecords);
-
-				// Update millis behind latest so it gets reported by the millisBehindLatest gauge
-				Long millisBehindLatest = getRecordsResult.getMillisBehindLatest();
-				if (millisBehindLatest != null) {
-					shardMetricsReporter.setMillisBehindLatest(millisBehindLatest);
-				}
-			} catch (ExpiredIteratorException eiEx) {
-				LOG.warn("Encountered an unexpected expired iterator {} for shard {};" +
-					" refreshing the iterator ...", shardItr, subscribedShard);
-
-				shardItr = getShardIterator(lastSequenceNum);
-
-				// sleep for the fetch interval before the next getRecords attempt with the refreshed iterator
-				if (fetchIntervalMillis != 0) {
-					Thread.sleep(fetchIntervalMillis);
-				}
-			}
+	private boolean filterDeaggregatedRecord(final UserRecord record) {
+		if (lastSequenceNum.isAggregated()) {

Review comment:
       It is replicating this behaviour: https://github.com/apache/flink/blob/release-1.10.1/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardConsumer.java#L198
   
   Say you have an aggregated message X with 6 child records and you stop consuming at subsequence 3, the last consumed sequence number would be:
   - sequence: X
   - sub-sequence: 3
   
   Therefore when you restart at X, you need to discard subsequence 1,2,3 and process 4,5,6.
   
   This method discards any previously processed subsequence numbers continuously, rather than upfront. This allows the logic to be pulled into the `ShardConsumer` from the `RecordPublisher` and support EFO/Polling without configuring multiple clients. As explained in the PR description, the code is simpler putting this check here at the expense of a slight overhead.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org