You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tzulitai <gi...@git.apache.org> on 2016/06/19 17:15:31 UTC

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

GitHub user tzulitai opened a pull request:

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

    [FLINK-3231][streaming-connectors] FlinkKinesisConsumer rework to handle Kinesis resharding

    This change attempts to solve 2 issues:
    1. [FLINK-3231](https://issues.apache.org/jira/browse/FLINK-3231): Handle Kinesis-side resharding.
    2. [FLINK-4020](https://issues.apache.org/jira/browse/FLINK-4020): Remove shard list querying from Kinesis consumer constructor.
    
    Some notes on the implementation:
    - All subtasks has a thread that continuously polls for changes in the Kinesis stream, and uses exponential backoff with jitter to try to even out the concurrent Kinesis client describeStream operations across subtasks. Continuously polling is necessary because there's currently no way to "signal" a subtask that it has a new shard it should be subscribing to.
    - A big change is that all subtasks run a fetcher that continues to poll for shards, even if the subtask initially didn't have shards to consume (before, a MAX_VALUE waterwark was sent out).
    - Apart from the unit tests, I've manually tested this with `ManualExactlyOnceWithStreamReshardingTest`. However, since the `FlinkKinesisProducer` currently has some problem that records are resent when Kinesis streams are resharded (thus not allowing the exactly-once test to pass at all), this manual test uses a normal event generator instead of a producer topology running the `FlinkKinesisProducer`.
    
    Since this PR introduces considerable rework on the Kinesis consumer, I'll wait until this is merged before submitting [FLINK-4080](https://issues.apache.org/jira/browse/FLINK-4080) & [FLINK-4019](https://issues.apache.org/jira/browse/FLINK-4019).

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

    $ git pull https://github.com/tzulitai/flink FLINK-3231

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

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

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

    This closes #2131
    
----
commit 378ec8177f1bfe91b459233a8ce02e9f988c61ab
Author: Gordon Tai <go...@vm5.com>
Date:   2016-06-08T10:46:02Z

    [FLINK-4020] Move shard list querying to open() for Kinesis consumer

commit 2c9f1304d5f6220fe36ad9d7833a506651f3fee6
Author: Gordon Tai <tz...@gmail.com>
Date:   2016-06-19T16:15:43Z

    [FLINK-3231] FlinkKinesisConsumer rework to handle Kinesis resharding

----


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68409331
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -159,42 +217,65 @@ public String getShardIterator(KinesisStreamShard shard, String shardIteratorTyp
     		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
     	}
     
    +	private List<KinesisStreamShard> getShardsOfStream(String streamName, String lastSeenShardId) throws InterruptedException {
    +		List<KinesisStreamShard> shardsOfStream = new ArrayList<>();
    +
    +		DescribeStreamResult describeStreamResult;
    +		do {
    +			describeStreamResult = describeStream(streamName, lastSeenShardId);
    +
    +			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
    +			for (Shard shard : shards) {
    +				shardsOfStream.add(new KinesisStreamShard(streamName, shard));
    +			}
    +
    +			if (shards.size() != 0) {
    +				lastSeenShardId = shards.get(shards.size() - 1).getShardId();
    +			}
    +		} while (describeStreamResult.getStreamDescription().isHasMoreShards());
    +
    +		return shardsOfStream;
    +	}
    +
     	/**
     	 * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess.
     	 *
    +	 * This method is using a "full jitter" approach described in
    +	 * <a href="http://google.com">https://www.awsarchitectureblog.com/2015/03/backoff.html</a>. This is necessary
    +	 * because concurrent calls will be made by all parallel subtask's {@link ShardDiscoverer}s. This jitter backoff
    +	 * approach will help distribute calls across the discoverers over time.
    +	 *
     	 * @param streamName the stream to describe
     	 * @param startShardId which shard to start with for this describe operation (earlier shard's infos will not appear in result)
     	 * @return the result of the describe stream operation
     	 */
    -	private DescribeStreamResult describeStream(String streamName, String startShardId) {
    +	private DescribeStreamResult describeStream(String streamName, String startShardId) throws InterruptedException {
     		final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest();
     		describeStreamRequest.setStreamName(streamName);
     		describeStreamRequest.setExclusiveStartShardId(startShardId);
     
     		DescribeStreamResult describeStreamResult = null;
     		String streamStatus = null;
    -		int remainingRetryTimes = Integer.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES)));
    -		long describeStreamBackoffTimeInMillis = Long.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF)));
     
    -		// Call DescribeStream, with backoff and retries (if we get LimitExceededException).
    -		while ((remainingRetryTimes >= 0) && (describeStreamResult == null)) {
    +		// Call DescribeStream, with full-jitter backoff (if we get LimitExceededException).
    +		Random seed = null;
    +		int attemptCount = 0;
    +		while (describeStreamResult == null) { // retry until we get a result
     			try {
     				describeStreamResult = kinesisClient.describeStream(describeStreamRequest);
     				streamStatus = describeStreamResult.getStreamDescription().getStreamStatus();
     			} catch (LimitExceededException le) {
    -				LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for "
    -					+ describeStreamBackoffTimeInMillis + " millis.");
    -				try {
    -					Thread.sleep(describeStreamBackoffTimeInMillis);
    -				} catch (InterruptedException ie) {
    -					LOG.debug("Stream " + streamName + " : Sleep  was interrupted ", ie);
    +				if (seed == null) {
    +					seed = new Random();
     				}
    +				long backoffMillis = fullJitterBackoff(
    +					describeStreamBaseBackoffMillis, describeStreamMaxBackoffMillis, describeStreamExpConstant, attemptCount++, seed);
    +				LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for "
    +					+ backoffMillis + " millis.");
    +				Thread.sleep(backoffMillis);
     			} catch (ResourceNotFoundException re) {
     				throw new RuntimeException("Error while getting stream details", re);
     			}
    -			remainingRetryTimes--;
     		}
     
     		if (streamStatus == null) {
    --- End diff --
    
    Actually, since we'll be retrying until we get a describeStreamResult now, this message and RuntimeException can be removed.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68594061
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -86,32 +144,25 @@ public KinesisProxy(Properties configProps) {
     	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
     	 * @return the batch of retrieved records
     	 */
    -	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
    +	@Override
    +	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
     		final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
     		getRecordsRequest.setShardIterator(shardIterator);
     		getRecordsRequest.setLimit(maxRecordsToGet);
     
     		GetRecordsResult getRecordsResult = null;
     
    -		int remainingRetryTimes = Integer.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES)));
    -		long describeStreamBackoffTimeInMillis = Long.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF)));
    -
    -		int i=0;
    -		while (i <= remainingRetryTimes && getRecordsResult == null) {
    +		int attempt = 0;
    +		while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) {
     			try {
     				getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
     			} catch (ProvisionedThroughputExceededException ex) {
    +				long backoffMillis = fullJitterBackoff(
    +					getRecordsBaseBackoffMillis, getRecordsMaxBackoffMillis, getRecordsExpConstant, attempt++);
     				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
    -					+ describeStreamBackoffTimeInMillis + " millis.");
    -				try {
    -					Thread.sleep(describeStreamBackoffTimeInMillis);
    -				} catch (InterruptedException interruptEx) {
    -					//
    -				}
    +					+ backoffMillis + " millis.");
    +				Thread.sleep(backoffMillis);
     			}
    -			i++;
     		}
     
     		if (getRecordsResult == null) {
    --- End diff --
    
    Not sure if throwing an exception here is a good idea.
    From the javadocs of `getRecord()` it says: "Note that if the shard has been closed, the shard iterator can't return more data and <a>GetRecords</a> returns <code>null</code>". This means that our code would fail each time some shards are closed?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69299251
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    --- End diff --
    
    Indeed :)


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I've also tested the last commit on the unit & manual tests.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger Thanks Robert!
    Sorry for the delay, got caught up with other things. Will push within the next 2~3 hrs.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68733201
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    --- End diff --
    
    @rmetzger double check: are we still adding configurable frequency with 30 second default for this?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68406625
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java ---
    @@ -61,19 +68,23 @@ public String getShardId() {
     	}
     
     	public String getStartingSequenceNumber() {
    -		return shard.getSequenceNumberRange().getStartingSequenceNumber();
    +		SequenceNumberRange sequenceNumberRange = shard.getSequenceNumberRange();
    +		return (sequenceNumberRange == null) ? null : sequenceNumberRange.getStartingSequenceNumber();
    --- End diff --
    
    In what cases is the sequence range null?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69298876
  
    --- Diff: docs/apis/streaming/connectors/kinesis.md ---
    @@ -60,10 +60,10 @@ to setup Kinesis streams. Make sure to create the appropriate IAM policy and use
     
     ### Kinesis Consumer
    --- End diff --
    
    Good point! I'll add this.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68410216
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java ---
    @@ -61,19 +68,23 @@ public String getShardId() {
     	}
     
     	public String getStartingSequenceNumber() {
    -		return shard.getSequenceNumberRange().getStartingSequenceNumber();
    +		SequenceNumberRange sequenceNumberRange = shard.getSequenceNumberRange();
    +		return (sequenceNumberRange == null) ? null : sequenceNumberRange.getStartingSequenceNumber();
    --- End diff --
    
    If this is the case, than I don't see the reason for these checks.
    In general, I have the feeling that the `KinesisStreamShard` is overly complicated. Maybe its easier to just have a method getShard() instead of proxying everything?


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I'm going to merge this once travis gives me green light ;)


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68582164
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    +
    +			// after startup, we wait util either shutdownFetcher() or stopWithError() is called
    +			fetcherShutdownLock.wait();
    --- End diff --
    
    Doesn't calling `wait()` release the lock on `fetcherShutdownLock`?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69285572
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    --- End diff --
    
    I think the + at the end of the string is wrong.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69300764
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    +					runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(),
    +					seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
     			}
     
    -			// make sure any asynchronous error is noticed
    -			Throwable error = this.error.get();
    -			if (error != null) {
    -				throw new Exception(error.getMessage(), error);
    +			shardConsumersExecutor.submit(
    +				new ShardConsumer<>(
    +					this,
    +					seededStateIndex,
    +					subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
    +					subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
    +		}
    +
    +		// ------------------------------------------------------------------------
    +
    +		// finally, start the infinite shard discovery and consumer launching loop;
    +		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
    +
    +		final long discoveryIntervalMillis = Long.valueOf(
    +			configProps.getProperty(
    +				KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS,
    +				Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
    +
    +		while (running) {
    +			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
    +
    +			for (KinesisStreamShard shard : newShardsDueToResharding) {
    +				// since there may be delay in discovering a new shard, all new shards due to
    +				// resharding should be read starting from the earliest record possible
    +				KinesisStreamShardState newShardState =
    +					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
    +				int newStateIndex = registerNewSubscribedShardState(newShardState);
    +
    +				if (LOG.isInfoEnabled()) {
    +					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
    +							"the shard from sequence number {} with ShardConsumer {}",
    +						runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(),
    +						newShardState.getLastProcessedSequenceNum(), newStateIndex);
    --- End diff --
    
    Oh I see. Ok!


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69129641
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    +					runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(),
    +					seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
     			}
     
    -			// make sure any asynchronous error is noticed
    -			Throwable error = this.error.get();
    -			if (error != null) {
    -				throw new Exception(error.getMessage(), error);
    +			shardConsumersExecutor.submit(
    +				new ShardConsumer<>(
    +					this,
    +					seededStateIndex,
    +					subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
    +					subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
    +		}
    +
    +		// ------------------------------------------------------------------------
    +
    +		// finally, start the infinite shard discovery and consumer launching loop;
    +		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
    +
    +		final long discoveryIntervalMillis = Long.valueOf(
    +			configProps.getProperty(
    +				KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS,
    +				Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
    +
    +		while (running) {
    +			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
    +
    +			for (KinesisStreamShard shard : newShardsDueToResharding) {
    +				// since there may be delay in discovering a new shard, all new shards due to
    +				// resharding should be read starting from the earliest record possible
    +				KinesisStreamShardState newShardState =
    +					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
    +				int newStateIndex = registerNewSubscribedShardState(newShardState);
    +
    +				if (LOG.isInfoEnabled()) {
    +					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
    +							"the shard from sequence number {} with ShardConsumer {}",
    +						runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(),
    +						newShardState.getLastProcessedSequenceNum(), newStateIndex);
    +				}
    +
    +				shardConsumersExecutor.submit(
    +					new ShardConsumer<>(
    +						this,
    +						newStateIndex,
    +						newShardState.getKinesisStreamShard(),
    +						newShardState.getLastProcessedSequenceNum()));
     			}
    -		} finally {
    -			for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -				if (consumerThread.isAlive()) {
    -					consumerThread.cancel();
    +
    +			// we also check if we are running here so that we won't start the discovery sleep
    +			// interval if the running flag was set to false during the middle of the while loop
    +			if (running && discoveryIntervalMillis != 0) {
    +				try {
    +					Thread.sleep(discoveryIntervalMillis);
    +				} catch (InterruptedException iex) {
    +					// the sleep may be interrupted by shutdownFetcher()
    --- End diff --
    
    I'm a bit concerned about this part here, it seems a bit floppy. I'm not sure if there's a better way to avoid start sleeping for the discovery interval when `running` is set to false by `shutdownFetcher()` mid-loop ...
    @rmetzger Please let me know what you think about this one.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I tested the code, also with a shard-merging:
    `aws kinesis merge-shards  --shard-to-merge shardId-000000000001 --adjacent-shard-to-merge shardId-000000000002 --stream-name flink-test`
    
    and everything worked nicely, the log statements were good.
    If you want to improve it a little bit, we should maybe log at debug level each time we discover new shards (just to show that everything is working as expected).
    
    The only thing missing are some minor documents, then, I think we are good to merge.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68411242
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java ---
    @@ -61,19 +68,23 @@ public String getShardId() {
     	}
     
     	public String getStartingSequenceNumber() {
    -		return shard.getSequenceNumberRange().getStartingSequenceNumber();
    +		SequenceNumberRange sequenceNumberRange = shard.getSequenceNumberRange();
    +		return (sequenceNumberRange == null) ? null : sequenceNumberRange.getStartingSequenceNumber();
    --- End diff --
    
    Okay, I agree!


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68560835
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(ShardDiscoverer.class);
    +
    +	/** This fetcher reference is used to add discovered shards to the pending shards queue */
    +	private final KinesisDataFetcher fetcherRef;
    +
    +	/** Kinesis proxy to retrieve shard lists from Kinesis */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/**
    +	 * The last seen shard of each stream. Since new Kinesis shards are always created in ascending ids (regardless of
    +	 * whether the new shard was a result of a shard split or merge), this state can be used when calling
    +	 * {@link KinesisProxyInterface#getShardList(Map)} to ignore shards we have already discovered before.
    +	 */
    +	private final Map<String,String> streamToLastSeenShard;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * Create a new shard discoverer.
    +	 *
    +	 * @param fetcherRef reference to the owning fetcher
    +	 */
    +	public ShardDiscoverer(KinesisDataFetcher<T> fetcherRef) {
    +		this(fetcherRef, KinesisProxy.create(fetcherRef.getConsumerConfiguration()), new HashMap<String, String>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected ShardDiscoverer(KinesisDataFetcher<T> fetcherRef,
    +							KinesisProxyInterface kinesis,
    +							Map<String,String> streamToLastSeenShard) {
    +		this.fetcherRef = checkNotNull(fetcherRef);
    +		this.kinesis = checkNotNull(kinesis);
    +		this.streamToLastSeenShard = checkNotNull(streamToLastSeenShard);
    +
    +		this.totalNumberOfConsumerSubtasks = fetcherRef.getSubtaskRuntimeContext().getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = fetcherRef.getSubtaskRuntimeContext().getIndexOfThisSubtask();
    +
    +		// we initially map the last seen shard of each subscribed stream to null;
    +		// the correct values will be set later on in the constructor
    +		for (String stream : fetcherRef.getSubscribedStreams()) {
    +			this.streamToLastSeenShard.put(stream, null);
    +		}
    +
    +		// if we are restoring from a checkpoint, the restored state should already be in the pending shards queue;
    +		// we iterate over the pending shards queue, and accordingly set the stream-to-last-seen-shard map
    +		if (fetcherRef.isRestoredFromCheckpoint()) {
    +			if (fetcherRef.getCurrentCountOfPendingShards() == 0) {
    +				throw new RuntimeException("Told to restore from checkpoint, but no shards found in discovered shards queue");
    +			}
    +
    +			for (KinesisStreamShardState shardState : fetcherRef.cloneCurrentPendingShards()) {
    +				String stream = shardState.getKinesisStreamShard().getStreamName();
    +				String shardId = shardState.getKinesisStreamShard().getShard().getShardId();
    +				if (!this.streamToLastSeenShard.containsKey(stream)) {
    +					throw new RuntimeException(
    +						"pendingShards queue contains a shard belonging to a stream that we are not subscribing to");
    +				} else {
    +					String lastSeenShardIdOfStream = this.streamToLastSeenShard.get(stream);
    +					// the existing shards in the queue may not be in ascending id order,
    +					// so we must exhaustively find the largest shard id of each stream
    +					if (lastSeenShardIdOfStream == null) {
    +						// if not previously set, simply put as the last seen shard id
    +						this.streamToLastSeenShard.put(stream, shardId);
    +					} else if (KinesisStreamShard.compareShardIds(shardId, lastSeenShardIdOfStream) > 0) {
    +						// override if we have found a shard with a greater shard id for the stream
    +						this.streamToLastSeenShard.put(stream, shardId);
    +					}
    +				}
    +			}
    +		}
    +
    +		// we always query for any new shards that may have been created while the Kinesis consumer was not running -
    +		// when we are starting fresh (not restoring from a checkpoint), this simply means all existing shards of streams
    +		// we are subscribing to are new shards; when we are restoring from checkpoint, any new shards due to Kinesis
    +		// resharding from the time of the checkpoint will be considered new shards.
    +
    +		SentinelSequenceNumber sentinelSequenceNumber;
    +		if (!fetcherRef.isRestoredFromCheckpoint()) {
    +			// if starting fresh, each new shard should start from the user-configured position
    +			sentinelSequenceNumber =
    +				KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(fetcherRef.getConsumerConfiguration());
    +		} else {
    +			// if restoring from checkpoint, the new shards due to Kinesis resharding should be read from earliest record
    +			sentinelSequenceNumber = SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM;
    +		}
    +
    +		try {
    +			// query for new shards that we haven't seen yet up to this point
    +			discoverNewShardsAndSetInitialStateTo(sentinelSequenceNumber.toString());
    +		} catch (InterruptedException iex) {
    +			fetcherRef.stopWithError(iex);
    +			return;
    +		}
    +
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastSeenShardEntry : streamToLastSeenShard.entrySet()) {
    +			if (streamToLastSeenShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastSeenShardEntry.getKey()).append(", ");
    +			}
    +		}
    +
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
    +		}
    +
    +		if (!hasShards) {
    +			fetcherRef.stopWithError(new RuntimeException(
    +				"No shards can be found for all subscribed streams: " + fetcherRef.getSubscribedStreams()));
    +		}
    +	}
    +
    +	@Override
    +	public void run() {
    +		try {
    +			while (isRunning()) {
    +				discoverNewShardsAndSetInitialStateTo(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.toString());
    +			}
    +		} catch (Throwable throwable) {
    +			fetcherRef.stopWithError(throwable);
    +		}
    +	}
    +
    +	private boolean isRunning() {
    +		return !Thread.interrupted();
    +	}
    +
    +	/**
    +	 * A utility function that does the following:
    +	 *
    +	 * 1. Find new shards for each stream that we haven't seen before
    +	 * 2. For each new shard, determine whether this consumer subtask should subscribe to them;
    +	 * 	  if yes, add the new shard to the discovered shards queue with a specified initial state (starting sequence num)
    +	 * 3. Update the stream-to-last-seen-shard map so that we won't get shards that we have already seen before
    +	 *    the next time this function is called
    +	 *
    +	 * @param initialState the initial state to assign to each new shard that this subtask should subscribe to
    +	 */
    +	private void discoverNewShardsAndSetInitialStateTo(String initialState) throws InterruptedException {
    +		GetShardListResult shardListResult = kinesis.getShardList(new HashMap<>(streamToLastSeenShard));
    --- End diff --
    
    Why are you creating a new HashMap here?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68578735
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    --- End diff --
    
    You are right, this thread continuously polls Kinesis. I was worried about the delay it would introduce, and how users would take the delay with it being a streaming source.
    But yes I agree it will be best to introduce configurable frequency, I was initially a bit unsure of the continuous polling myself. We can add description about this delay of discovering new shards after resharding in the consumer documentation.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68582306
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    --- End diff --
    
    Okay. I'm still not done with the review, so my statement might not make much sense:
    We could also trigger a special shard discovery when a reading thread hits a deleted / unavailable shard. This way, we can reduce the delay for our users.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @tzulitai: I'm still not done with the review. I hope I can get it done in the next two hours.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68402254
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -159,42 +217,65 @@ public String getShardIterator(KinesisStreamShard shard, String shardIteratorTyp
     		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
     	}
     
    +	private List<KinesisStreamShard> getShardsOfStream(String streamName, String lastSeenShardId) throws InterruptedException {
    +		List<KinesisStreamShard> shardsOfStream = new ArrayList<>();
    +
    +		DescribeStreamResult describeStreamResult;
    +		do {
    +			describeStreamResult = describeStream(streamName, lastSeenShardId);
    +
    +			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
    +			for (Shard shard : shards) {
    +				shardsOfStream.add(new KinesisStreamShard(streamName, shard));
    +			}
    +
    +			if (shards.size() != 0) {
    +				lastSeenShardId = shards.get(shards.size() - 1).getShardId();
    +			}
    +		} while (describeStreamResult.getStreamDescription().isHasMoreShards());
    +
    +		return shardsOfStream;
    +	}
    +
     	/**
     	 * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess.
     	 *
    +	 * This method is using a "full jitter" approach described in
    +	 * <a href="http://google.com">https://www.awsarchitectureblog.com/2015/03/backoff.html</a>. This is necessary
    --- End diff --
    
    Why is this linking to google.com?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68562955
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    +
    +			// after startup, we wait util either shutdownFetcher() or stopWithError() is called
    +			fetcherShutdownLock.wait();
    --- End diff --
    
    I think the `shutdownFetcher()` can never call notifyAll because this method is never releasing the lock on `fetcherShutdownLock`.
    
    What happens instead is that Flink is interrupting the thread holding the lock. This interrupt will cause an InterruptedException, which is not catched at all.
    
    I don't think this waiting mechanism is necessary here at all. Maybe we can use the `awaitTermination()` for that (but maybe there is an even better solution available ... I'll continue with the review)


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    Hi @rmetzger ,
    I've addressed your initial comments with the last commit. No urgent hurry on the remaining review, please take your time :) Thanks!


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69274234
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java ---
    @@ -68,10 +97,30 @@
     	//  Default configuration values
     	// ------------------------------------------------------------------------
     
    -	public static final int DEFAULT_STREAM_DESCRIBE_RETRY_TIMES = 3;
    +	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L;
    +
    +	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L;
    +
    +	public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
    +
    +	public static final int DEFAULT_SHARD_GETRECORDS_MAX = 100;
    +
    +	public static final int DEFAULT_SHARD_GETRECORDS_RETRIES = 3;
    +
    +	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE = 300L;
    +
    +	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX = 1000L;
    +
    +	public static final double DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
    +
    +	public static final int DEFAULT_SHARD_GETITERATOR_RETRIES = 3;
    +
    +	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE = 300L;
    +
    +	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX = 1000L;
     
    -	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF = 1000L;
    +	public static final double DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
     
    -	public static final int DEFAULT_SHARD_RECORDS_PER_GET = 100;
    +	public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L;
    --- End diff --
    
    Okay. 10 seconds is okay as well.



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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68404148
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -159,42 +217,65 @@ public String getShardIterator(KinesisStreamShard shard, String shardIteratorTyp
     		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
     	}
     
    +	private List<KinesisStreamShard> getShardsOfStream(String streamName, String lastSeenShardId) throws InterruptedException {
    +		List<KinesisStreamShard> shardsOfStream = new ArrayList<>();
    +
    +		DescribeStreamResult describeStreamResult;
    +		do {
    +			describeStreamResult = describeStream(streamName, lastSeenShardId);
    +
    +			List<Shard> shards = describeStreamResult.getStreamDescription().getShards();
    +			for (Shard shard : shards) {
    +				shardsOfStream.add(new KinesisStreamShard(streamName, shard));
    +			}
    +
    +			if (shards.size() != 0) {
    +				lastSeenShardId = shards.get(shards.size() - 1).getShardId();
    +			}
    +		} while (describeStreamResult.getStreamDescription().isHasMoreShards());
    +
    +		return shardsOfStream;
    +	}
    +
     	/**
     	 * Get metainfo for a Kinesis stream, which contains information about which shards this Kinesis stream possess.
     	 *
    +	 * This method is using a "full jitter" approach described in
    +	 * <a href="http://google.com">https://www.awsarchitectureblog.com/2015/03/backoff.html</a>. This is necessary
    +	 * because concurrent calls will be made by all parallel subtask's {@link ShardDiscoverer}s. This jitter backoff
    +	 * approach will help distribute calls across the discoverers over time.
    +	 *
     	 * @param streamName the stream to describe
     	 * @param startShardId which shard to start with for this describe operation (earlier shard's infos will not appear in result)
     	 * @return the result of the describe stream operation
     	 */
    -	private DescribeStreamResult describeStream(String streamName, String startShardId) {
    +	private DescribeStreamResult describeStream(String streamName, String startShardId) throws InterruptedException {
     		final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest();
     		describeStreamRequest.setStreamName(streamName);
     		describeStreamRequest.setExclusiveStartShardId(startShardId);
     
     		DescribeStreamResult describeStreamResult = null;
     		String streamStatus = null;
    -		int remainingRetryTimes = Integer.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES)));
    -		long describeStreamBackoffTimeInMillis = Long.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF)));
     
    -		// Call DescribeStream, with backoff and retries (if we get LimitExceededException).
    -		while ((remainingRetryTimes >= 0) && (describeStreamResult == null)) {
    +		// Call DescribeStream, with full-jitter backoff (if we get LimitExceededException).
    +		Random seed = null;
    +		int attemptCount = 0;
    +		while (describeStreamResult == null) { // retry until we get a result
     			try {
     				describeStreamResult = kinesisClient.describeStream(describeStreamRequest);
     				streamStatus = describeStreamResult.getStreamDescription().getStreamStatus();
     			} catch (LimitExceededException le) {
    -				LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for "
    -					+ describeStreamBackoffTimeInMillis + " millis.");
    -				try {
    -					Thread.sleep(describeStreamBackoffTimeInMillis);
    -				} catch (InterruptedException ie) {
    -					LOG.debug("Stream " + streamName + " : Sleep  was interrupted ", ie);
    +				if (seed == null) {
    +					seed = new Random();
     				}
    +				long backoffMillis = fullJitterBackoff(
    +					describeStreamBaseBackoffMillis, describeStreamMaxBackoffMillis, describeStreamExpConstant, attemptCount++, seed);
    +				LOG.warn("Got LimitExceededException when describing stream " + streamName + ". Backing off for "
    +					+ backoffMillis + " millis.");
    +				Thread.sleep(backoffMillis);
     			} catch (ResourceNotFoundException re) {
     				throw new RuntimeException("Error while getting stream details", re);
     			}
    -			remainingRetryTimes--;
     		}
     
     		if (streamStatus == null) {
    --- End diff --
    
    The `RuntimeException` below has a typo.
    `Can't get stream info from ____ after`.
    
    Also, I wonder where the number 3 is coming from.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68409807
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java ---
    @@ -61,19 +68,23 @@ public String getShardId() {
     	}
     
     	public String getStartingSequenceNumber() {
    -		return shard.getSequenceNumberRange().getStartingSequenceNumber();
    +		SequenceNumberRange sequenceNumberRange = shard.getSequenceNumberRange();
    +		return (sequenceNumberRange == null) ? null : sequenceNumberRange.getStartingSequenceNumber();
    --- End diff --
    
    I added these null checks because in the tests, fake `KinesisStreamShards` may be created without giving them a SequenceNumberRange / HashNumberRange etc. if it wasn't relevant to the test. Perhaps I should revert these null check changes, and always give all values for the fake `KinesisStreamShard`s?


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    Okay, sounds good.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68558779
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java ---
    @@ -0,0 +1,29 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.proxy;
    +
    +import com.amazonaws.services.kinesis.model.GetRecordsResult;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +
    +import java.util.Map;
    +
    +public interface KinesisProxyInterface {
    --- End diff --
    
    Can you add a little bit of javadocs to the class and the methods?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68731349
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -155,55 +199,79 @@ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
     	 * @param startingSeqNum the sequence number that the iterator will start from
     	 * @return the shard iterator
     	 */
    +	@Override
     	public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
    -		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
    +		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShard().getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
    --- End diff --
    
    Okay. Actually, the 5 rps limit for `getShardIterator()` is per shard. We'll only have one thread making this call for each shard, so the rate limit won't be a problem for us. But as explained above, we should still retry + fail hard to let the user know if they need to adjust other non-Flink consumers.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68722233
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    --- End diff --
    
    I wonder if we can use the main thread for the shard discovery? Its sitting there waiting only.
    This way, we could get rid of the `shardDiscovererAndSubscriberExecutor` thread pool.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68399467
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -122,29 +173,36 @@ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
     	}
     
     	/**
    -	 * Get the list of shards associated with multiple Kinesis streams
    +	 * Get the complete shard list of multiple Kinesis streams.
     	 *
    -	 * @param streamNames the list of Kinesis streams
    -	 * @return a list of {@link KinesisStreamShard}s
    +	 * @param streamNames Kinesis streams to retrieve the shard list for
    +	 * @return shard list result
     	 */
    -	public List<KinesisStreamShard> getShardList(List<String> streamNames) {
    -		List<KinesisStreamShard> shardList = new ArrayList<>();
    +	public GetShardListResult getShardList(List<String> streamNames) throws InterruptedException {
    +		GetShardListResult result = new GetShardListResult();
     
     		for (String stream : streamNames) {
    -			DescribeStreamResult describeStreamResult;
    -			String lastSeenShardId = null;
    +			result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, null));
    +		}
    +		return result;
    +	}
     
    -			do {
    -				describeStreamResult = describeStream(stream, lastSeenShardId);
    +	/**
    +	 * Get shard list of multiple Kinesis streams, ignoring the
    +	 * shards of each streambefore a specified last seen shard id.
    +	 *
    +	 * @param streamNamesWithLastSeenShardIds a map with stream as key, and last seen shard id as value
    +	 * @return shard list result
    +	 */
    +	public GetShardListResult getShardList(Map<String,String> streamNamesWithLastSeenShardIds) throws InterruptedException {
    --- End diff --
    
    Missing `@Override` annotation


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I'm done with my first full review. I'll test the code tomorrow.
    Let me know if you need help addressing my comments.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68729145
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -155,55 +199,79 @@ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
     	 * @param startingSeqNum the sequence number that the iterator will start from
     	 * @return the shard iterator
     	 */
    +	@Override
     	public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
    -		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
    +		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShard().getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
    --- End diff --
    
    Yes, I think we should have retries there as well. In particular during start up, many parallel Flink instances will query AWS for the shard iterator.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68583275
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    +
    +			// after startup, we wait util either shutdownFetcher() or stopWithError() is called
    +			fetcherShutdownLock.wait();
    --- End diff --
    
    You are right. Sorry.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68726797
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    --- End diff --
    
    I agree with
    > Simply put, I don't think we can soley rely on the event of encountering a closed shard to let the new shards get fully picked up by the correct subtasks.
    
    My idea was to trigger an additional discovery when we hit a closed shard.
    We could for example discover shards every minute OR when we hit a closed shard. But you are right, this won't solve the issue with other parallel instances.
    Lets keep it as is for now.
    Thank you for the explanation.
    
    Lets keep the assignment from `ShardDiscoverer#isShouldSubscribeTo()`. 


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger tagging just to make sure you're notified of this PR :)
    When will you be free to review? Just for my own time allocation for when to continue working on the Kinesis connector. If there's anything majorly wrong with the implementation explained in the above comment, please let me know and I'll try to address them before effort on a detailed review.
    Thanks in advance!


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger
    Thanks a lot for your helpful review :) I should be able to address your comments before tomorrow so we can start testing it.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    Thank you for the quick fix. I hope I can take a look tonight. Otherwise, I'll look at it early next week.
    Thanks a lot for addressing my comments to quickly.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69130874
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/KinesisConfigConstants.java ---
    @@ -68,10 +97,30 @@
     	//  Default configuration values
     	// ------------------------------------------------------------------------
     
    -	public static final int DEFAULT_STREAM_DESCRIBE_RETRY_TIMES = 3;
    +	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L;
    +
    +	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L;
    +
    +	public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
    +
    +	public static final int DEFAULT_SHARD_GETRECORDS_MAX = 100;
    +
    +	public static final int DEFAULT_SHARD_GETRECORDS_RETRIES = 3;
    +
    +	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_BASE = 300L;
    +
    +	public static final long DEFAULT_SHARD_GETRECORDS_BACKOFF_MAX = 1000L;
    +
    +	public static final double DEFAULT_SHARD_GETRECORDS_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
    +
    +	public static final int DEFAULT_SHARD_GETITERATOR_RETRIES = 3;
    +
    +	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_BASE = 300L;
    +
    +	public static final long DEFAULT_SHARD_GETITERATOR_BACKOFF_MAX = 1000L;
     
    -	public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF = 1000L;
    +	public static final double DEFAULT_SHARD_GETITERATOR_BACKOFF_EXPONENTIAL_CONSTANT = 1.5;
     
    -	public static final int DEFAULT_SHARD_RECORDS_PER_GET = 100;
    +	public static final long DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS = 10000L;
    --- End diff --
    
    I'm using 10s for default discovery interval here. I tested it and the originally suggested 30s seemed a bit too long as a default, IMHO. Can change it back to 30s if you think it's more appropriate.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68399454
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -122,29 +173,36 @@ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
     	}
     
     	/**
    -	 * Get the list of shards associated with multiple Kinesis streams
    +	 * Get the complete shard list of multiple Kinesis streams.
     	 *
    -	 * @param streamNames the list of Kinesis streams
    -	 * @return a list of {@link KinesisStreamShard}s
    +	 * @param streamNames Kinesis streams to retrieve the shard list for
    +	 * @return shard list result
     	 */
    -	public List<KinesisStreamShard> getShardList(List<String> streamNames) {
    -		List<KinesisStreamShard> shardList = new ArrayList<>();
    +	public GetShardListResult getShardList(List<String> streamNames) throws InterruptedException {
    --- End diff --
    
    Missing `@Override` annotation


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68409932
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java ---
    @@ -61,19 +68,23 @@ public String getShardId() {
     	}
     
     	public String getStartingSequenceNumber() {
    -		return shard.getSequenceNumberRange().getStartingSequenceNumber();
    +		SequenceNumberRange sequenceNumberRange = shard.getSequenceNumberRange();
    +		return (sequenceNumberRange == null) ? null : sequenceNumberRange.getStartingSequenceNumber();
    --- End diff --
    
    The `Shard`s returned from actual Kinesis will never have null values for these.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger Thank you for your review. I hope I've addressed your last comments with the last commit.
    
    For the documentation, I added a bit more apart from the threading model: 1) enabling checkpointing, mostly borrowed from the Kafka documentation, and 2) information on how the consumer internally uses the Kinesis APIs so that users can make sense of any limit warnings they are getting in the logs.
    
    Please let me know if there's anything else to address!


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger Thank you! No problem, I can wait and address the current comments meanwhile.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger 
    No problem, thank you! I'll reply to your comments after you finish reviewing :)


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68408986
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,481 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		if (LOG.isInfoEnabled()) {
    +			LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +		}
    +		shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +		// after this point we will start fetching data from Kinesis and update internal state,
    +		// so we check that we are running for the last time before continuing
    +		if (!running) {
    +			return;
    +		}
    +
    +		if (LOG.isInfoEnabled()) {
    +			LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +		}
    +		shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    +
    +		while (running) {
    +			// once either shutdownFetcher() or stopWithError()
    +			// is called, we will escape this loop
    +		}
    --- End diff --
    
    Yup this is really bad ... will fix!


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68564678
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    --- End diff --
    
    It seems that this thread is continuously querying the Kinesis APIs. The only way to slow these requests down is by the jittered backoff.
    I think we should query the kinesis api for reshards at a configurable frequency, probably by default only every 30 seconds or so.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68397478
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,481 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		if (LOG.isInfoEnabled()) {
    +			LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +		}
    +		shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +		// after this point we will start fetching data from Kinesis and update internal state,
    +		// so we check that we are running for the last time before continuing
    +		if (!running) {
    +			return;
    +		}
    +
    +		if (LOG.isInfoEnabled()) {
    +			LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +		}
    +		shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    +
    +		while (running) {
    +			// once either shutdownFetcher() or stopWithError()
    +			// is called, we will escape this loop
    +		}
    --- End diff --
    
    This loop is extremely inefficient. Is a very busy wait loop (one CPU core will constantly burn cycles on this loop).


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68930725
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    --- End diff --
    
    Yes.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68711841
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -155,55 +199,79 @@ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
     	 * @param startingSeqNum the sequence number that the iterator will start from
     	 * @return the shard iterator
     	 */
    +	@Override
     	public String getShardIterator(KinesisStreamShard shard, String shardIteratorType, String startingSeqNum) {
    -		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
    +		return kinesisClient.getShardIterator(shard.getStreamName(), shard.getShard().getShardId(), shardIteratorType, startingSeqNum).getShardIterator();
    --- End diff --
    
    @rmetzger This method doesn't do the `ProvisionedThroughputExceededException` check / retries like the `getRecords()` method. Perhaps we should be checking that + fail hard after retrying too, as it also has a 5 requests per second limit? Reasoning is the same as my reply to your comment on the `getRecords()` method.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68413466
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -206,4 +287,9 @@ private DescribeStreamResult describeStream(String streamName, String startShard
     			throw new RuntimeException("Stream is not Active or Updating");
    --- End diff --
    
    I agree! We shouldn't fail hard here.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69300586
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    +					runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(),
    +					seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
     			}
     
    -			// make sure any asynchronous error is noticed
    -			Throwable error = this.error.get();
    -			if (error != null) {
    -				throw new Exception(error.getMessage(), error);
    +			shardConsumersExecutor.submit(
    +				new ShardConsumer<>(
    +					this,
    +					seededStateIndex,
    +					subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
    +					subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
    +		}
    +
    +		// ------------------------------------------------------------------------
    +
    +		// finally, start the infinite shard discovery and consumer launching loop;
    +		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
    +
    +		final long discoveryIntervalMillis = Long.valueOf(
    +			configProps.getProperty(
    +				KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS,
    +				Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
    +
    +		while (running) {
    +			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
    +
    +			for (KinesisStreamShard shard : newShardsDueToResharding) {
    +				// since there may be delay in discovering a new shard, all new shards due to
    +				// resharding should be read starting from the earliest record possible
    +				KinesisStreamShardState newShardState =
    +					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
    +				int newStateIndex = registerNewSubscribedShardState(newShardState);
    +
    +				if (LOG.isInfoEnabled()) {
    +					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
    +							"the shard from sequence number {} with ShardConsumer {}",
    +						runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(),
    +						newShardState.getLastProcessedSequenceNum(), newStateIndex);
    --- End diff --
    
    I would add a LOG.debug statement everytime we call `discoverNewShardsToSubscribe()`.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    The ManualExactlyOnceTest and ManualExactlyOnceWithStreamReshardingTest is a bit buggy, fixing with a follow-up commit ...


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I'm not done with the review yet. I hope I find some time over the weekend to continue with it.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68414043
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -86,32 +141,28 @@ public KinesisProxy(Properties configProps) {
     	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
     	 * @return the batch of retrieved records
     	 */
    -	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
    +	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
     		final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
     		getRecordsRequest.setShardIterator(shardIterator);
     		getRecordsRequest.setLimit(maxRecordsToGet);
     
     		GetRecordsResult getRecordsResult = null;
     
    -		int remainingRetryTimes = Integer.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES)));
    -		long describeStreamBackoffTimeInMillis = Long.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF)));
    -
    -		int i=0;
    -		while (i <= remainingRetryTimes && getRecordsResult == null) {
    +		Random seed = null;
    +		int attempt = 0;
    +		while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) {
     			try {
     				getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
     			} catch (ProvisionedThroughputExceededException ex) {
    -				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
    -					+ describeStreamBackoffTimeInMillis + " millis.");
    -				try {
    -					Thread.sleep(describeStreamBackoffTimeInMillis);
    -				} catch (InterruptedException interruptEx) {
    -					//
    +				if (seed == null) {
    +					seed = new Random();
    --- End diff --
    
    Nope. I'll change to create it as a static class member. `getRecords` and `describeStream` can actually share the random when calling `fullJitterBackoff`.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68564082
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    --- End diff --
    
    Why do we need a separate thread for this?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68582645
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/ShardDiscoverer.java ---
    @@ -0,0 +1,215 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.internals;
    +
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
    +import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.apache.flink.util.Preconditions.checkNotNull;
    +
    +/**
    + * This runnable is in charge of discovering new shards that a fetcher should subscribe to.
    + * It is submitted to {@link KinesisDataFetcher#shardDiscovererAndSubscriberExecutor} and continuously runs until the
    + * fetcher is closed. Whenever it discovers a new shard that should be subscribed to, the shard is added to the
    + * {@link KinesisDataFetcher#pendingShards} queue with initial state, i.e. where in the new shard we should start
    + * consuming from.
    + */
    +public class ShardDiscoverer<T> implements Runnable {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(ShardDiscoverer.class);
    +
    +	/** This fetcher reference is used to add discovered shards to the pending shards queue */
    +	private final KinesisDataFetcher fetcherRef;
    +
    +	/** Kinesis proxy to retrieve shard lists from Kinesis */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/**
    +	 * The last seen shard of each stream. Since new Kinesis shards are always created in ascending ids (regardless of
    +	 * whether the new shard was a result of a shard split or merge), this state can be used when calling
    +	 * {@link KinesisProxyInterface#getShardList(Map)} to ignore shards we have already discovered before.
    +	 */
    +	private final Map<String,String> streamToLastSeenShard;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * Create a new shard discoverer.
    +	 *
    +	 * @param fetcherRef reference to the owning fetcher
    +	 */
    +	public ShardDiscoverer(KinesisDataFetcher<T> fetcherRef) {
    +		this(fetcherRef, KinesisProxy.create(fetcherRef.getConsumerConfiguration()), new HashMap<String, String>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected ShardDiscoverer(KinesisDataFetcher<T> fetcherRef,
    +							KinesisProxyInterface kinesis,
    +							Map<String,String> streamToLastSeenShard) {
    +		this.fetcherRef = checkNotNull(fetcherRef);
    +		this.kinesis = checkNotNull(kinesis);
    +		this.streamToLastSeenShard = checkNotNull(streamToLastSeenShard);
    +
    +		this.totalNumberOfConsumerSubtasks = fetcherRef.getSubtaskRuntimeContext().getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = fetcherRef.getSubtaskRuntimeContext().getIndexOfThisSubtask();
    +
    +		// we initially map the last seen shard of each subscribed stream to null;
    +		// the correct values will be set later on in the constructor
    +		for (String stream : fetcherRef.getSubscribedStreams()) {
    +			this.streamToLastSeenShard.put(stream, null);
    +		}
    +
    +		// if we are restoring from a checkpoint, the restored state should already be in the pending shards queue;
    +		// we iterate over the pending shards queue, and accordingly set the stream-to-last-seen-shard map
    +		if (fetcherRef.isRestoredFromCheckpoint()) {
    +			if (fetcherRef.getCurrentCountOfPendingShards() == 0) {
    +				throw new RuntimeException("Told to restore from checkpoint, but no shards found in discovered shards queue");
    +			}
    +
    +			for (KinesisStreamShardState shardState : fetcherRef.cloneCurrentPendingShards()) {
    +				String stream = shardState.getKinesisStreamShard().getStreamName();
    +				String shardId = shardState.getKinesisStreamShard().getShard().getShardId();
    +				if (!this.streamToLastSeenShard.containsKey(stream)) {
    +					throw new RuntimeException(
    +						"pendingShards queue contains a shard belonging to a stream that we are not subscribing to");
    +				} else {
    +					String lastSeenShardIdOfStream = this.streamToLastSeenShard.get(stream);
    +					// the existing shards in the queue may not be in ascending id order,
    +					// so we must exhaustively find the largest shard id of each stream
    +					if (lastSeenShardIdOfStream == null) {
    +						// if not previously set, simply put as the last seen shard id
    +						this.streamToLastSeenShard.put(stream, shardId);
    +					} else if (KinesisStreamShard.compareShardIds(shardId, lastSeenShardIdOfStream) > 0) {
    +						// override if we have found a shard with a greater shard id for the stream
    +						this.streamToLastSeenShard.put(stream, shardId);
    +					}
    +				}
    +			}
    +		}
    +
    +		// we always query for any new shards that may have been created while the Kinesis consumer was not running -
    +		// when we are starting fresh (not restoring from a checkpoint), this simply means all existing shards of streams
    +		// we are subscribing to are new shards; when we are restoring from checkpoint, any new shards due to Kinesis
    +		// resharding from the time of the checkpoint will be considered new shards.
    +
    +		SentinelSequenceNumber sentinelSequenceNumber;
    +		if (!fetcherRef.isRestoredFromCheckpoint()) {
    +			// if starting fresh, each new shard should start from the user-configured position
    +			sentinelSequenceNumber =
    +				KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(fetcherRef.getConsumerConfiguration());
    +		} else {
    +			// if restoring from checkpoint, the new shards due to Kinesis resharding should be read from earliest record
    +			sentinelSequenceNumber = SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM;
    +		}
    +
    +		try {
    +			// query for new shards that we haven't seen yet up to this point
    +			discoverNewShardsAndSetInitialStateTo(sentinelSequenceNumber.toString());
    +		} catch (InterruptedException iex) {
    +			fetcherRef.stopWithError(iex);
    +			return;
    +		}
    +
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastSeenShardEntry : streamToLastSeenShard.entrySet()) {
    +			if (streamToLastSeenShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastSeenShardEntry.getKey()).append(", ");
    +			}
    +		}
    +
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
    +		}
    +
    +		if (!hasShards) {
    +			fetcherRef.stopWithError(new RuntimeException(
    +				"No shards can be found for all subscribed streams: " + fetcherRef.getSubscribedStreams()));
    +		}
    +	}
    +
    +	@Override
    +	public void run() {
    +		try {
    +			while (isRunning()) {
    +				discoverNewShardsAndSetInitialStateTo(SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.toString());
    +			}
    +		} catch (Throwable throwable) {
    +			fetcherRef.stopWithError(throwable);
    +		}
    +	}
    +
    +	private boolean isRunning() {
    +		return !Thread.interrupted();
    +	}
    +
    +	/**
    +	 * A utility function that does the following:
    +	 *
    +	 * 1. Find new shards for each stream that we haven't seen before
    +	 * 2. For each new shard, determine whether this consumer subtask should subscribe to them;
    +	 * 	  if yes, add the new shard to the discovered shards queue with a specified initial state (starting sequence num)
    +	 * 3. Update the stream-to-last-seen-shard map so that we won't get shards that we have already seen before
    +	 *    the next time this function is called
    +	 *
    +	 * @param initialState the initial state to assign to each new shard that this subtask should subscribe to
    +	 */
    +	private void discoverNewShardsAndSetInitialStateTo(String initialState) throws InterruptedException {
    +		GetShardListResult shardListResult = kinesis.getShardList(new HashMap<>(streamToLastSeenShard));
    --- End diff --
    
    Mistake, absolutely unnecessary. Thanks for noticing!


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68400657
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -122,29 +173,36 @@ public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
     	}
     
     	/**
    -	 * Get the list of shards associated with multiple Kinesis streams
    +	 * Get the complete shard list of multiple Kinesis streams.
     	 *
    -	 * @param streamNames the list of Kinesis streams
    -	 * @return a list of {@link KinesisStreamShard}s
    +	 * @param streamNames Kinesis streams to retrieve the shard list for
    +	 * @return shard list result
     	 */
    -	public List<KinesisStreamShard> getShardList(List<String> streamNames) {
    -		List<KinesisStreamShard> shardList = new ArrayList<>();
    +	public GetShardListResult getShardList(List<String> streamNames) throws InterruptedException {
    --- End diff --
    
    We can probably drop this unused method.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69296319
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    +					runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(),
    +					seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
     			}
     
    -			// make sure any asynchronous error is noticed
    -			Throwable error = this.error.get();
    -			if (error != null) {
    -				throw new Exception(error.getMessage(), error);
    +			shardConsumersExecutor.submit(
    +				new ShardConsumer<>(
    +					this,
    +					seededStateIndex,
    +					subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
    +					subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
    +		}
    +
    +		// ------------------------------------------------------------------------
    +
    +		// finally, start the infinite shard discovery and consumer launching loop;
    +		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
    +
    +		final long discoveryIntervalMillis = Long.valueOf(
    +			configProps.getProperty(
    +				KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS,
    +				Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
    +
    +		while (running) {
    +			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
    +
    +			for (KinesisStreamShard shard : newShardsDueToResharding) {
    +				// since there may be delay in discovering a new shard, all new shards due to
    +				// resharding should be read starting from the earliest record possible
    +				KinesisStreamShardState newShardState =
    +					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
    +				int newStateIndex = registerNewSubscribedShardState(newShardState);
    +
    +				if (LOG.isInfoEnabled()) {
    +					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
    +							"the shard from sequence number {} with ShardConsumer {}",
    +						runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(),
    +						newShardState.getLastProcessedSequenceNum(), newStateIndex);
    +				}
    +
    +				shardConsumersExecutor.submit(
    +					new ShardConsumer<>(
    +						this,
    +						newStateIndex,
    +						newShardState.getKinesisStreamShard(),
    +						newShardState.getLastProcessedSequenceNum()));
     			}
    -		} finally {
    -			for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -				if (consumerThread.isAlive()) {
    -					consumerThread.cancel();
    +
    +			// we also check if we are running here so that we won't start the discovery sleep
    +			// interval if the running flag was set to false during the middle of the while loop
    +			if (running && discoveryIntervalMillis != 0) {
    +				try {
    +					Thread.sleep(discoveryIntervalMillis);
    +				} catch (InterruptedException iex) {
    +					// the sleep may be interrupted by shutdownFetcher()
    --- End diff --
    
    I think the approach is fine. running will be set to false, then the interrupt to leave the sleep.
    Flink will also start interrupting the thread if its not reacting.
    Also, once the next sleep has finished it will check the running flag again, leaving the loop.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    @rmetzger
    Just pushed the changes to address comments + rebase on the exactly-once fix / user-agent fix.
    Sorry, had some trouble with the rebasing and took more time than I expected.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68399389
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/IKinesisProxy.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.proxy;
    +
    +import com.amazonaws.services.kinesis.model.GetRecordsResult;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public interface IKinesisProxy {
    --- End diff --
    
    The `I` prefix is not commonly used within Flink.
    I would suggest `KinesisProxyInterface` as a name here.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68727843
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -86,32 +144,25 @@ public KinesisProxy(Properties configProps) {
     	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
     	 * @return the batch of retrieved records
     	 */
    -	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
    +	@Override
    +	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
     		final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
     		getRecordsRequest.setShardIterator(shardIterator);
     		getRecordsRequest.setLimit(maxRecordsToGet);
     
     		GetRecordsResult getRecordsResult = null;
     
    -		int remainingRetryTimes = Integer.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES)));
    -		long describeStreamBackoffTimeInMillis = Long.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF)));
    -
    -		int i=0;
    -		while (i <= remainingRetryTimes && getRecordsResult == null) {
    +		int attempt = 0;
    +		while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) {
     			try {
     				getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
     			} catch (ProvisionedThroughputExceededException ex) {
    +				long backoffMillis = fullJitterBackoff(
    +					getRecordsBaseBackoffMillis, getRecordsMaxBackoffMillis, getRecordsExpConstant, attempt++);
     				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
    -					+ describeStreamBackoffTimeInMillis + " millis.");
    -				try {
    -					Thread.sleep(describeStreamBackoffTimeInMillis);
    -				} catch (InterruptedException interruptEx) {
    -					//
    -				}
    +					+ backoffMillis + " millis.");
    +				Thread.sleep(backoffMillis);
     			}
    -			i++;
     		}
     
     		if (getRecordsResult == null) {
    --- End diff --
    
    Thank you for the good explanation. I agree to keep it as it is.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68730716
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    --- End diff --
    
    Good idea! Yes, I think we can use the main thread. I'll probably integrate the work done in `ShardDiscoverer` and `ShardSubscriber` into `KinesisDataFetcher#runFetcher()`. `KinesisDataFetcher#shutdownFetcher()` should also be much simpler with this change.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68403594
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -86,32 +141,28 @@ public KinesisProxy(Properties configProps) {
     	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
     	 * @return the batch of retrieved records
     	 */
    -	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
    +	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
     		final GetRecordsRequest getRecordsRequest = new GetRecordsRequest();
     		getRecordsRequest.setShardIterator(shardIterator);
     		getRecordsRequest.setLimit(maxRecordsToGet);
     
     		GetRecordsResult getRecordsResult = null;
     
    -		int remainingRetryTimes = Integer.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_RETRIES, Integer.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_RETRY_TIMES)));
    -		long describeStreamBackoffTimeInMillis = Long.valueOf(
    -			configProps.getProperty(KinesisConfigConstants.CONFIG_STREAM_DESCRIBE_BACKOFF, Long.toString(KinesisConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF)));
    -
    -		int i=0;
    -		while (i <= remainingRetryTimes && getRecordsResult == null) {
    +		Random seed = null;
    +		int attempt = 0;
    +		while (attempt <= getRecordsMaxAttempts && getRecordsResult == null) {
     			try {
     				getRecordsResult = kinesisClient.getRecords(getRecordsRequest);
     			} catch (ProvisionedThroughputExceededException ex) {
    -				LOG.warn("Got ProvisionedThroughputExceededException. Backing off for "
    -					+ describeStreamBackoffTimeInMillis + " millis.");
    -				try {
    -					Thread.sleep(describeStreamBackoffTimeInMillis);
    -				} catch (InterruptedException interruptEx) {
    -					//
    +				if (seed == null) {
    +					seed = new Random();
    --- End diff --
    
    Is there a reason for creating a new RNG with every `getRecords` call?


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I'll review your change now.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68399431
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -86,32 +141,28 @@ public KinesisProxy(Properties configProps) {
     	 * @param maxRecordsToGet the maximum amount of records to retrieve for this batch
     	 * @return the batch of retrieved records
     	 */
    -	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) {
    +	public GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException {
    --- End diff --
    
    Missing `@Override` annotation


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68582684
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyInterface.java ---
    @@ -0,0 +1,29 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.proxy;
    +
    +import com.amazonaws.services.kinesis.model.GetRecordsResult;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +
    +import java.util.Map;
    +
    +public interface KinesisProxyInterface {
    --- End diff --
    
    Sure, no problem.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68412025
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -206,4 +287,9 @@ private DescribeStreamResult describeStream(String streamName, String startShard
     			throw new RuntimeException("Stream is not Active or Updating");
    --- End diff --
    
    Actually, should we really be throwing a RuntimeException if a single stream is not in ACTIVE or UPDATING status? Say the consumer is to fetch 3 streams, and only 1 is found to be CREATING / DELETING. Perhaps we should treat this case the same as "can only find shards for some of the streams", and a warning log here will do?
    
    There's a check that stops the consumer in the new `ShardDiscoverer` if no shards can be initially found at all to read from anyway.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69300348
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    +					runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(),
    +					seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
     			}
     
    -			// make sure any asynchronous error is noticed
    -			Throwable error = this.error.get();
    -			if (error != null) {
    -				throw new Exception(error.getMessage(), error);
    +			shardConsumersExecutor.submit(
    +				new ShardConsumer<>(
    +					this,
    +					seededStateIndex,
    +					subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
    +					subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
    +		}
    +
    +		// ------------------------------------------------------------------------
    +
    +		// finally, start the infinite shard discovery and consumer launching loop;
    +		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
    +
    +		final long discoveryIntervalMillis = Long.valueOf(
    +			configProps.getProperty(
    +				KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS,
    +				Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
    +
    +		while (running) {
    +			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
    +
    +			for (KinesisStreamShard shard : newShardsDueToResharding) {
    +				// since there may be delay in discovering a new shard, all new shards due to
    +				// resharding should be read starting from the earliest record possible
    +				KinesisStreamShardState newShardState =
    +					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
    +				int newStateIndex = registerNewSubscribedShardState(newShardState);
    +
    +				if (LOG.isInfoEnabled()) {
    +					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
    +							"the shard from sequence number {} with ShardConsumer {}",
    +						runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(),
    +						newShardState.getLastProcessedSequenceNum(), newStateIndex);
    --- End diff --
    
    @rmetzger We're already logging here when new shards are discovered. Should I still add the debug level log you mentioned, perhaps in `KinesisDataFetcher#discoverNewShardsToSubscribe()`?


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    Okay, I'll merge the exactly once fix now.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68403918
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java ---
    @@ -206,4 +287,9 @@ private DescribeStreamResult describeStream(String streamName, String startShard
     			throw new RuntimeException("Stream is not Active or Updating");
    --- End diff --
    
    Can you put the actual stream status into the exception?


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68579220
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,156 +17,489 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    -import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +
    +import java.util.concurrent.BlockingQueue;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.LinkedBlockingQueue;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher runs several threads to accomplish
    + * the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) pending shards for subscription, and 2) last processed sequence numbers of
    + * each subscribed shard. All operations on the states in multiple threads should only be done using the handler methods
    + * provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, multiple threads in the fetcher using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
    +
    +	/** Executor service to run the {@link ShardDiscoverer} and {@link ShardSubscriber} */
    +	private final ExecutorService shardDiscovererAndSubscriberExecutor;
    +
    +	/** Executor service to run {@link ShardConsumer}s to consumer Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, String> assignedShardsWithStartingSequenceNum;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	/**
    +	 * Blocking queue for newly discovered shards, with their states, that this fetcher should consume.
    +	 * The {@link ShardDiscoverer} will add shards with initial position as state to this queue as shards are discovered,
    +	 * while the {@link ShardSubscriber} polls this queue to start subscribing to the new discovered shards.
    +	 */
    +	private final BlockingQueue<KinesisStreamShardState> pendingShards;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The shard
    +	 * subscriber will add to this list as it polls pending shards. Shard consumer threads update the last processed
    +	 * sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple threads will be performing operations on this list, all operations must be wrapped in
    +	 * synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose, all threads must use
    +	 * the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#addAndStartConsumingNewSubscribedShard(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, String)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(Object, int, String)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** This flag is set to true if the fetcher is provided a non-null and non-empty restored state */
    +	private final boolean isRestoredFromCheckpoint;
    +
    +	/** Reference to the first error thrown by any of the created threads */
     	private final AtomicReference<Throwable> error;
     
    +	/**
    +	 *  Lock used by atomic operations to startup / shutdown the fetcher, preventing indeterminate behaviour of
    +	 *  creating and shutting down resources. Also, {@link Object#wait()} is called on this lock after the startup
    +	 *  process completes in {@link KinesisDataFetcher#runFetcher()}. We wake from the wait only when
    +	 *  {@link KinesisDataFetcher#shutdownFetcher()} has been called to execute the shutdown process.
    +	 */
    +	private final Object fetcherShutdownLock = new Object();
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param restoredState state of subcribed shards that the fetcher should restore to
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							Map<KinesisStreamShard, String> restoredState,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,configProps,
    +			restoredState,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedBlockingQueue<KinesisStreamShardState>(),
    +			new LinkedList<KinesisStreamShardState>());
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								Map<KinesisStreamShard, String> restoredState,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedBlockingQueue<KinesisStreamShardState> pendingShardsQueue,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.toString());
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +
    +		this.error = error;
    +		this.pendingShards = pendingShardsQueue;
    +		this.subscribedShardsState = subscribedShardsState;
    +
    +		this.shardDiscovererAndSubscriberExecutor =
    +			createShardDiscovererAndSubscriberThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
    +
    +		this.isRestoredFromCheckpoint = (restoredState != null && restoredState.entrySet().size() != 0);
    +
    +		// if there is state to restore from last checkpoint, we seed them as initially discovered shards
    +		if (isRestoredFromCheckpoint) {
    +			seedPendingShardsWithRestoredState(restoredState, this.pendingShards);
     		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, String sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    +	public void runFetcher() throws Exception {
    +
    +		// atomic operation to startup the fetcher and create the shard discoverer and subscriber.
    +		synchronized (fetcherShutdownLock) {
    +
    +			// this flag will be false if the shutdown procedure was
    +			// executed first; if so, we return without doing anything.
    +			if (!running) {
    +				return;
    +			}
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard discoverer ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardDiscoverer<>(this));
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} is starting the shard subscriber ...", runtimeContext.getIndexOfThisSubtask());
    +			}
    +			shardDiscovererAndSubscriberExecutor.submit(new ShardSubscriber<>(this));
    --- End diff --
    
    Perhaps we can simply integrate the functionality of this thread into `ShardDiscoverer`? Once new shards we should be subscribing to is discovered, we open up a new `ShardConsumer` for it. This we also won't need the `pendingShards` queue.


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

[GitHub] flink issue #2131: [FLINK-3231][streaming-connectors] FlinkKinesisConsumer r...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2131
  
    I'll rebase this after #2180 is merged to reflect in this PR the exactly-once fix, and commit altogether with the changes to address the remaining comments.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69299145
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java ---
    @@ -17,157 +17,553 @@
     
     package org.apache.flink.streaming.connectors.kinesis.internals;
     
    +import org.apache.flink.api.common.functions.RuntimeContext;
     import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer;
    +import org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
     import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState;
     import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber;
     import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy;
    +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface;
     import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil;
     import org.apache.flink.util.InstantiationUtil;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
     import java.io.IOException;
    -import java.util.Map;
    -import java.util.HashMap;
    +
    +import java.util.LinkedList;
     import java.util.List;
    -import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Map;
     import java.util.Properties;
    +import java.util.Set;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.ThreadFactory;
    +import java.util.concurrent.atomic.AtomicLong;
     import java.util.concurrent.atomic.AtomicReference;
     
     import static org.apache.flink.util.Preconditions.checkNotNull;
     
     /**
    - * A Kinesis Data Fetcher that consumes data from a specific set of Kinesis shards.
    - * The fetcher spawns a single thread for connection to each shard.
    + * A KinesisDataFetcher is responsible for fetching data from multiple Kinesis shards. Each parallel subtask instantiates
    + * and runs a single fetcher throughout the subtask's lifetime. The fetcher accomplishes the following:
    + * <ul>
    + *     <li>1. continuously poll Kinesis to discover shards that the subtask should subscribe to. The subscribed subset
    + *     		  of shards, including future new shards, is non-overlapping across subtasks (no two subtasks will be
    + *     		  subscribed to the same shard) and determinate across subtask restores (the subtask will always subscribe
    + *     		  to the same subset of shards even after restoring)</li>
    + *     <li>2. decide where in each discovered shard should the fetcher start subscribing to</li>
    + *     <li>3. subscribe to shards by creating a single thread for each shard</li>
    + * </ul>
    + *
    + * <p>The fetcher manages two states: 1) last seen shard ids of each subscribed stream (used for continuous shard discovery),
    + * and 2) last processed sequence numbers of each subscribed shard. Since operations on the second state will be performed
    + * by multiple threads, these operations should only be done using the handler methods provided in this class.
      */
    -public class KinesisDataFetcher {
    +public class KinesisDataFetcher<T> {
     
     	private static final Logger LOG = LoggerFactory.getLogger(KinesisDataFetcher.class);
     
    -	/** Config properties for the Flink Kinesis Consumer */
    +	// ------------------------------------------------------------------------
    +	//  Consumer-wide settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Configuration properties for the Flink Kinesis Consumer */
     	private final Properties configProps;
     
    -	/** The name of the consumer task that this fetcher was instantiated */
    -	private final String taskName;
    +	/** The list of Kinesis streams that the consumer is subscribing to */
    +	private final List<String> streams;
    +
    +	/**
    +	 * The deserialization schema we will be using to convert Kinesis records to Flink objects.
    +	 * Note that since this might not be thread-safe, {@link ShardConsumer}s using this must
    +	 * clone a copy using {@link KinesisDataFetcher#getClonedDeserializationSchema()}.
    +	 */
    +	private final KinesisDeserializationSchema<T> deserializationSchema;
    +
    +	// ------------------------------------------------------------------------
    +	//  Subtask-specific settings
    +	// ------------------------------------------------------------------------
    +
    +	/** Runtime context of the subtask that this fetcher was created in */
    +	private final RuntimeContext runtimeContext;
    +
    +	private final int totalNumberOfConsumerSubtasks;
    +
    +	private final int indexOfThisConsumerSubtask;
    +
    +	/**
    +	 * This flag should be set by {@link FlinkKinesisConsumer} using
    +	 * {@link KinesisDataFetcher#setIsRestoringFromFailure(boolean)}
    +	 */
    +	private boolean isRestoredFromFailure;
    +
    +	// ------------------------------------------------------------------------
    +	//  Executor services to run created threads
    +	// ------------------------------------------------------------------------
     
    -	/** Information of the shards that this fetcher handles, along with the sequence numbers that they should start from */
    -	private HashMap<KinesisStreamShard, SequenceNumber> assignedShardsWithStartingSequenceNum;
    +	/** Executor service to run {@link ShardConsumer}s to consume Kinesis shards */
    +	private final ExecutorService shardConsumersExecutor;
     
    -	/** Reference to the thread that executed run() */
    -	private volatile Thread mainThread;
    +	// ------------------------------------------------------------------------
    +	//  Managed state, accessed and updated across multiple threads
    +	// ------------------------------------------------------------------------
     
    -	/** Reference to the first error thrown by any of the spawned shard connection threads */
    +	/** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in.
    +	 * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called.
    +	 */
    +	private final Map<String, String> subscribedStreamsToLastDiscoveredShardIds;
    +
    +	/**
    +	 * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher
    +	 * will add new subscribed shard states to this list as it discovers new shards. {@link ShardConsumer} threads update
    +	 * the last processed sequence number of subscribed shards as they fetch and process records.
    +	 *
    +	 * <p>Note that since multiple {@link ShardConsumer} threads will be performing operations on this list, all operations
    +	 * must be wrapped in synchronized blocks on the {@link KinesisDataFetcher#checkpointLock} lock. For this purpose,
    +	 * all threads must use the following thread-safe methods this class provides to operate on this list:
    +	 * <ul>
    +	 *     <li>{@link KinesisDataFetcher#registerNewSubscribedShardState(KinesisStreamShardState)}</li>
    +	 *     <li>{@link KinesisDataFetcher#updateState(int, SequenceNumber)}</li>
    +	 *     <li>{@link KinesisDataFetcher#emitRecordAndUpdateState(T, int, SequenceNumber)}</li>
    +	 * </ul>
    +	 */
    +	private final List<KinesisStreamShardState> subscribedShardsState;
    +
    +	private final SourceFunction.SourceContext<T> sourceContext;
    +
    +	/** Checkpoint lock, also used to synchronize operations on subscribedShardsState */
    +	private final Object checkpointLock;
    +
    +	/** Reference to the first error thrown by any of the {@link ShardConsumer} threads */
     	private final AtomicReference<Throwable> error;
     
    +	/** The Kinesis proxy that the fetcher will be using to discover new shards */
    +	private final KinesisProxyInterface kinesis;
    +
    +	/** Thread that executed runFetcher() */
    +	private Thread mainThread;
    +
     	private volatile boolean running = true;
     
     	/**
    -	 * Creates a new Kinesis Data Fetcher for the specified set of shards
    +	 * Creates a Kinesis Data Fetcher.
     	 *
    -	 * @param assignedShards the shards that this fetcher will pull data from
    -	 * @param configProps the configuration properties of this Flink Kinesis Consumer
    -	 * @param taskName the task name of this consumer task
    +	 * @param streams the streams to subscribe to
    +	 * @param sourceContext context of the source function
    +	 * @param runtimeContext this subtask's runtime context
    +	 * @param configProps the consumer configuration properties
    +	 * @param deserializationSchema deserialization schema
     	 */
    -	public KinesisDataFetcher(List<KinesisStreamShard> assignedShards, Properties configProps, String taskName) {
    +	public KinesisDataFetcher(List<String> streams,
    +							SourceFunction.SourceContext<T> sourceContext,
    +							RuntimeContext runtimeContext,
    +							Properties configProps,
    +							KinesisDeserializationSchema<T> deserializationSchema) {
    +		this(streams,
    +			sourceContext,
    +			sourceContext.getCheckpointLock(),
    +			runtimeContext,
    +			configProps,
    +			deserializationSchema,
    +			new AtomicReference<Throwable>(),
    +			new LinkedList<KinesisStreamShardState>(),
    +			createInitialSubscribedStreamsToLastDiscoveredShardsState(streams),
    +			KinesisProxy.create(configProps));
    +	}
    +
    +	/** This constructor is exposed for testing purposes */
    +	protected KinesisDataFetcher(List<String> streams,
    +								SourceFunction.SourceContext<T> sourceContext,
    +								Object checkpointLock,
    +								RuntimeContext runtimeContext,
    +								Properties configProps,
    +								KinesisDeserializationSchema<T> deserializationSchema,
    +								AtomicReference<Throwable> error,
    +								LinkedList<KinesisStreamShardState> subscribedShardsState,
    +								HashMap<String, String> subscribedStreamsToLastDiscoveredShardIds,
    +								KinesisProxyInterface kinesis) {
    +		this.streams = checkNotNull(streams);
     		this.configProps = checkNotNull(configProps);
    -		this.assignedShardsWithStartingSequenceNum = new HashMap<>();
    -		for (KinesisStreamShard shard : assignedShards) {
    -			assignedShardsWithStartingSequenceNum.put(shard, SentinelSequenceNumber.SENTINEL_SEQUENCE_NUMBER_NOT_SET.get());
    -		}
    -		this.taskName = taskName;
    -		this.error = new AtomicReference<>();
    +		this.sourceContext = checkNotNull(sourceContext);
    +		this.checkpointLock = checkNotNull(checkpointLock);
    +		this.runtimeContext = checkNotNull(runtimeContext);
    +		this.totalNumberOfConsumerSubtasks = runtimeContext.getNumberOfParallelSubtasks();
    +		this.indexOfThisConsumerSubtask = runtimeContext.getIndexOfThisSubtask();
    +		this.deserializationSchema = checkNotNull(deserializationSchema);
    +		this.kinesis = checkNotNull(kinesis);
    +
    +		this.error = checkNotNull(error);
    +		this.subscribedShardsState = checkNotNull(subscribedShardsState);
    +		this.subscribedStreamsToLastDiscoveredShardIds = checkNotNull(subscribedStreamsToLastDiscoveredShardIds);
    +
    +		this.shardConsumersExecutor =
    +			createShardConsumersThreadPool(runtimeContext.getTaskNameWithSubtasks());
     	}
     
     	/**
    -	 * Advance a shard's starting sequence number to a specified value
    +	 * Starts the fetcher. After starting the fetcher, it can only
    +	 * be stopped by calling {@link KinesisDataFetcher#shutdownFetcher()}.
     	 *
    -	 * @param streamShard the shard to perform the advance on
    -	 * @param sequenceNum the sequence number to advance to
    +	 * @throws Exception the first error or exception thrown by the fetcher or any of the threads created by the fetcher.
     	 */
    -	public void advanceSequenceNumberTo(KinesisStreamShard streamShard, SequenceNumber sequenceNum) {
    -		if (!assignedShardsWithStartingSequenceNum.containsKey(streamShard)) {
    -			throw new IllegalArgumentException("Can't advance sequence number on a shard we are not going to read.");
    -		}
    -		assignedShardsWithStartingSequenceNum.put(streamShard, sequenceNum);
    -	}
    -
    -	public <T> void run(SourceFunction.SourceContext<T> sourceContext,
    -						KinesisDeserializationSchema<T> deserializationSchema,
    -						HashMap<KinesisStreamShard, SequenceNumber> lastSequenceNums) throws Exception {
    +	public void runFetcher() throws Exception {
     
    -		if (assignedShardsWithStartingSequenceNum == null || assignedShardsWithStartingSequenceNum.size() == 0) {
    -			throw new IllegalArgumentException("No shards set to read for this fetcher");
    +		// check that we are running before proceeding
    +		if (!running) {
    +			return;
     		}
     
     		this.mainThread = Thread.currentThread();
     
    -		LOG.info("Reading from shards " + assignedShardsWithStartingSequenceNum);
    +		// ------------------------------------------------------------------------
    +		//  Procedures before starting the infinite while loop:
    +		// ------------------------------------------------------------------------
    +
    +		//  1. query for any new shards that may have been created while the Kinesis consumer was not running,
    +		//     and register them to the subscribedShardState list.
    +		List<KinesisStreamShard> newShardsCreatedWhileNotRunning = discoverNewShardsToSubscribe();
    +		for (KinesisStreamShard shard : newShardsCreatedWhileNotRunning) {
    +			// the starting state for new shards created while the consumer wasn't running depends on whether or not
    +			// we are starting fresh (not restoring from a checkpoint); when we are starting fresh, this simply means
    +			// all existing shards of streams we are subscribing to are new shards; when we are restoring from checkpoint,
    +			// any new shards due to Kinesis resharding from the time of the checkpoint will be considered new shards.
    +			SentinelSequenceNumber startingStateForNewShard = (isRestoredFromFailure)
    +				? SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM
    +				: KinesisConfigUtil.getInitialPositionAsSentinelSequenceNumber(configProps);
     
    -		// create a thread for each individual shard
    -		ArrayList<ShardConsumerThread<?>> consumerThreads = new ArrayList<>(assignedShardsWithStartingSequenceNum.size());
    -		for (Map.Entry<KinesisStreamShard, SequenceNumber> assignedShard : assignedShardsWithStartingSequenceNum.entrySet()) {
    -			ShardConsumerThread<T> thread = new ShardConsumerThread<>(this, configProps, assignedShard.getKey(),
    -				assignedShard.getValue(), sourceContext, InstantiationUtil.clone(deserializationSchema), lastSequenceNums);
    -			thread.setName(String.format("ShardConsumer - %s - %s/%s",
    -				taskName, assignedShard.getKey().getStreamName() ,assignedShard.getKey().getShardId()));
    -			thread.setDaemon(true);
    -			consumerThreads.add(thread);
    +			if (LOG.isInfoEnabled()) {
    +				String logFormat = (isRestoredFromFailure)
    +					? "Subtask {} will be seeded with initial shard {}, starting state set as sequence number {}"
    +					: "Subtask {} will be seeded with new shard {} that was created while the consumer wasn't" +
    +						"running due to failure, starting state set as sequence number {}";
    +
    +				LOG.info(logFormat, runtimeContext.getIndexOfThisSubtask(), shard.toString(), startingStateForNewShard.get());
    +			}
    +			registerNewSubscribedShardState(new KinesisStreamShardState(shard, startingStateForNewShard.get()));
     		}
     
    -		// check that we are viable for running for the last time before starting threads
    -		if (!running) {
    -			return;
    +		//  2. check that there is at least one shard in the subscribed streams to consume from (can be done by
    +		//     checking if at least one value in subscribedStreamsToLastDiscoveredShardIds is not null)
    +		boolean hasShards = false;
    +		StringBuilder streamsWithNoShardsFound = new StringBuilder();
    +		for (Map.Entry<String, String> streamToLastDiscoveredShardEntry : subscribedStreamsToLastDiscoveredShardIds.entrySet()) {
    +			if (streamToLastDiscoveredShardEntry.getValue() != null) {
    +				hasShards = true;
    +			} else {
    +				streamsWithNoShardsFound.append(streamToLastDiscoveredShardEntry.getKey()).append(", ");
    +			}
     		}
     
    -		for (ShardConsumerThread<?> shardConsumer : consumerThreads) {
    -			LOG.info("Starting thread {}", shardConsumer.getName());
    -			shardConsumer.start();
    +		if (streamsWithNoShardsFound.length() != 0 && LOG.isWarnEnabled()) {
    +			LOG.warn("Subtask {} has failed to find any shards for the following subscribed streams: {}",
    +				indexOfThisConsumerSubtask, streamsWithNoShardsFound.toString());
     		}
     
    -		// wait until all consumer threads are done, or until the fetcher is aborted, or until
    -		// an error occurred in one of the consumer threads
    -		try {
    -			boolean consumersStillRunning = true;
    -			while (running && error.get() == null && consumersStillRunning) {
    -				try {
    -					// wait for the consumer threads. if an error occurs, we are interrupted
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumerThread.join();
    -					}
    +		if (!hasShards) {
    +			throw new RuntimeException("No shards can be found for all subscribed streams: " + streams);
    +		}
     
    -					// check if there are consumer threads still running
    -					consumersStillRunning = false;
    -					for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -						consumersStillRunning = consumersStillRunning | consumerThread.isAlive();
    -					}
    -				} catch (InterruptedException e) {
    -					// ignore
    -				}
    +		//  3. start consuming any shard state we already have in the subscribedShardState up to this point; the
    +		//     subscribedShardState may already be seeded with values due to step 1., or explicitly added by the
    +		//     consumer using a restored state checkpoint
    +		for (int seededStateIndex = 0; seededStateIndex < subscribedShardsState.size(); seededStateIndex++) {
    +			KinesisStreamShardState seededShardState = subscribedShardsState.get(seededStateIndex);
    +
    +			if (LOG.isInfoEnabled()) {
    +				LOG.info("Subtask {} will start consuming seeded shard {} from sequence number {} with ShardConsumer {}" +
    +					runtimeContext.getIndexOfThisSubtask(), seededShardState.getKinesisStreamShard().toString(),
    +					seededShardState.getLastProcessedSequenceNum(), seededStateIndex);
     			}
     
    -			// make sure any asynchronous error is noticed
    -			Throwable error = this.error.get();
    -			if (error != null) {
    -				throw new Exception(error.getMessage(), error);
    +			shardConsumersExecutor.submit(
    +				new ShardConsumer<>(
    +					this,
    +					seededStateIndex,
    +					subscribedShardsState.get(seededStateIndex).getKinesisStreamShard(),
    +					subscribedShardsState.get(seededStateIndex).getLastProcessedSequenceNum()));
    +		}
    +
    +		// ------------------------------------------------------------------------
    +
    +		// finally, start the infinite shard discovery and consumer launching loop;
    +		// we will escape from this loop only when shutdownFetcher() or stopWithError() is called
    +
    +		final long discoveryIntervalMillis = Long.valueOf(
    +			configProps.getProperty(
    +				KinesisConfigConstants.CONFIG_SHARD_DISCOVERY_INTERVAL_MILLIS,
    +				Long.toString(KinesisConfigConstants.DEFAULT_SHARD_DISCOVERY_INTERVAL_MILLIS)));
    +
    +		while (running) {
    +			List<KinesisStreamShard> newShardsDueToResharding = discoverNewShardsToSubscribe();
    +
    +			for (KinesisStreamShard shard : newShardsDueToResharding) {
    +				// since there may be delay in discovering a new shard, all new shards due to
    +				// resharding should be read starting from the earliest record possible
    +				KinesisStreamShardState newShardState =
    +					new KinesisStreamShardState(shard, SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get());
    +				int newStateIndex = registerNewSubscribedShardState(newShardState);
    +
    +				if (LOG.isInfoEnabled()) {
    +					LOG.info("Subtask {} has discovered a new shard {} due to resharding, and will start consuming " +
    +							"the shard from sequence number {} with ShardConsumer {}",
    +						runtimeContext.getIndexOfThisSubtask(), newShardState.getKinesisStreamShard().toString(),
    +						newShardState.getLastProcessedSequenceNum(), newStateIndex);
    +				}
    +
    +				shardConsumersExecutor.submit(
    +					new ShardConsumer<>(
    +						this,
    +						newStateIndex,
    +						newShardState.getKinesisStreamShard(),
    +						newShardState.getLastProcessedSequenceNum()));
     			}
    -		} finally {
    -			for (ShardConsumerThread<?> consumerThread : consumerThreads) {
    -				if (consumerThread.isAlive()) {
    -					consumerThread.cancel();
    +
    +			// we also check if we are running here so that we won't start the discovery sleep
    +			// interval if the running flag was set to false during the middle of the while loop
    +			if (running && discoveryIntervalMillis != 0) {
    +				try {
    +					Thread.sleep(discoveryIntervalMillis);
    +				} catch (InterruptedException iex) {
    +					// the sleep may be interrupted by shutdownFetcher()
    --- End diff --
    
    > Flink will also start interrupting the thread if its not reacting.
    I see! Okay, thanks, let's leave it as it is.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r69274851
  
    --- Diff: docs/apis/streaming/connectors/kinesis.md ---
    @@ -60,10 +60,10 @@ to setup Kinesis streams. Make sure to create the appropriate IAM policy and use
     
     ### Kinesis Consumer
    --- End diff --
    
    I think we should also mention the threading model of the Kinesis consumer in the documentation.
    
    Users should know that each parallel Flink instance will constantly query Kinesis for shards. So if a user has 5 shards, but runs Flink with a parallelism of 50, there will be 50 threads constantly querying Kinesis.
    
    Also, we should explain that there will always be one thread per shard running.


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

[GitHub] flink pull request #2131: [FLINK-3231][streaming-connectors] FlinkKinesisCon...

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2131#discussion_r68400596
  
    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/IKinesisProxy.java ---
    @@ -0,0 +1,31 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kinesis.proxy;
    +
    +import com.amazonaws.services.kinesis.model.GetRecordsResult;
    +import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public interface IKinesisProxy {
    +	GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException;
    +	GetShardListResult getShardList(List<String> streamNames) throws InterruptedException;
    --- End diff --
    
    This method is unused.


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