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

    https://github.com/apache/flink/pull/1911#discussion_r60756732
  
    --- Diff: 
flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java
 ---
    @@ -0,0 +1,481 @@
    +/*
    + * 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;
    +
    +import com.amazonaws.regions.Regions;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.runtime.state.CheckpointListener;
    +import 
org.apache.flink.streaming.api.checkpoint.CheckpointedAsynchronously;
    +import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
    +import org.apache.flink.streaming.api.watermark.Watermark;
    +import 
org.apache.flink.streaming.connectors.kinesis.config.KinesisConfigConstants;
    +import 
org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
    +import 
org.apache.flink.streaming.connectors.kinesis.config.CredentialProviderType;
    +import 
org.apache.flink.streaming.connectors.kinesis.config.InitialPosition;
    +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.proxy.KinesisProxy;
    +import 
org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper;
    +import 
org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
    +import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
    +import org.apache.flink.streaming.util.serialization.DeserializationSchema;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.ArrayList;
    +import java.util.Properties;
    +
    +import static com.google.common.base.Preconditions.checkArgument;
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +/**
    + * The Flink Kinesis Consumer is a parallel streaming data source that 
pulls data from multiple AWS Kinesis streams
    + * within the same AWS service region. Each instance of the consumer is 
responsible for fetching data records from
    + * one or more Kinesis shards.
    + *
    + * <p>To leverage Flink's checkpointing mechanics for exactly-once 
streaming processing guarantees, the Flink Kinesis
    + * consumer is implemented with the AWS Java SDK, instead of the 
officially recommended AWS Kinesis Client Library, for
    + * low-level control on the management of stream state. The Flink Kinesis 
Connector also supports setting the initial
    + * starting points of Kinesis streams, namely TRIM_HORIZON and LATEST:</p>
    + *
    + * <ul>
    + * </ul>
    + *
    + * <p><b>NOTE:</b> The current implementation does not correctly handle 
resharding of AWS Kinesis streams.</p>
    + * <p><b>NOTE:</b> Since Kinesis and Kafka share many common abstractions, 
the implementation is heavily based on
    + * the Flink Kafka Consumer.</p>
    + *
    + * @param <T> the type of data emitted
    + */
    +public class FlinkKinesisConsumer<T> extends RichParallelSourceFunction<T>
    +   implements CheckpointListener, 
CheckpointedAsynchronously<HashMap<KinesisStreamShard, String>>, 
ResultTypeQueryable<T> {
    +
    +   private static final long serialVersionUID = 1L;
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(FlinkKinesisConsumer.class);
    +
    +   // 
------------------------------------------------------------------------
    +   //  Consumer properties
    +   // 
------------------------------------------------------------------------
    +
    +   /** The complete list of shards */
    +   private final List<KinesisStreamShard> shards;
    +
    +   /** Properties to parametrize settings such as AWS service region, 
initial position in stream,
    +    * shard list retrieval behaviours, etc */
    +   private final Properties configProps;
    +
    +   /** User supplied deseriliazation schema to convert Kinesis byte 
messages to Flink objects */
    +   private final KinesisDeserializationSchema<T> deserializer;
    +
    +   // 
------------------------------------------------------------------------
    +   //  Runtime state
    +   // 
------------------------------------------------------------------------
    +
    +   /** Per-task fetcher for Kinesis data records, where each fetcher pulls 
data from one or more Kinesis shards */
    +   private transient KinesisDataFetcher fetcher;
    +
    +   /** The sequence numbers of the last fetched data records from Kinesis 
by this task */
    +   private transient HashMap<KinesisStreamShard, String> lastSequenceNums;
    +
    +   /** The sequence numbers to restore to upon restore from failure */
    +   private transient HashMap<KinesisStreamShard, String> 
sequenceNumsToRestore;
    +
    +   private volatile boolean hasAssignedShards;
    +
    +   private volatile boolean running = true;
    +
    +
    +   // 
------------------------------------------------------------------------
    +   //  Constructors
    +   // 
------------------------------------------------------------------------
    +
    +   /**
    +    * Creates a new Flink Kinesis Consumer.
    +    *
    +    * <p>The AWS credentials to be used, AWS region of the Kinesis 
streams, initial position to start streaming
    +    * from are configured with a {@link Properties} instance.</p>
    +    *
    +    * @param stream
    +    *           The single AWS Kinesis stream to read from.
    +    * @param deserializer
    +    *           The deserializer used to convert raw bytes of Kinesis 
records to Java objects (without key).
    +    * @param configProps
    +    *           The properties used to configure AWS credentials, AWS 
region, and initial starting position.
    +    */
    +   public FlinkKinesisConsumer(String stream, DeserializationSchema<T> 
deserializer, Properties configProps) {
    +           this(stream, new 
KinesisDeserializationSchemaWrapper<T>(deserializer), configProps);
    +   }
    +
    +   /**
    +    * Creates a new Flink Kinesis Consumer.
    +    *
    +    * <p>The AWS credentials to be used, AWS region of the Kinesis 
streams, initial position to start streaming
    +    * from are configured with a {@link Properties} instance.</p>
    +    *
    +    * @param stream
    +    *           The single AWS Kinesis stream to read from.
    +    * @param deserializer
    +    *           The keyed deserializer used to convert raw bytes of 
Kinesis records to Java objects.
    +    * @param configProps
    +    *           The properties used to configure AWS credentials, AWS 
region, and initial starting position.
    +    */
    +   public FlinkKinesisConsumer(String stream, 
KinesisDeserializationSchema<T> deserializer, Properties configProps) {
    +           this(Collections.singletonList(stream), deserializer, 
configProps);
    +   }
    +
    +   /**
    +    * Creates a new Flink Kinesis Consumer.
    +    *
    +    * <p>The AWS credentials to be used, AWS region of the Kinesis 
streams, initial position to start streaming
    +    * from are configured with a {@link Properties} instance.</p>
    +    *
    +    * @param streams
    +    *           The AWS Kinesis streams to read from.
    +    * @param deserializer
    +    *           The keyed deserializer used to convert raw bytes of 
Kinesis records to Java objects.
    +    * @param configProps
    +    *           The properties used to configure AWS credentials, AWS 
region, and initial starting position.
    +    */
    +   public FlinkKinesisConsumer(List<String> streams, 
KinesisDeserializationSchema<T> deserializer, Properties configProps) {
    +           checkNotNull(streams, "streams can not be null");
    +
    +           this.configProps = checkNotNull(configProps, "configProps can 
not be null");
    +
    +           // check the configuration properties for any conflicting 
settings
    +           validatePropertiesConfig(this.configProps);
    +
    +           this.deserializer = checkNotNull(deserializer, "deserializer 
can not be null");
    +
    +           this.shards = new 
KinesisProxy(configProps).getShardList(streams);
    --- End diff --
    
    My main concern when deciding to implement getting complete shard list at 
the client is due to the Amazon's limitation of the DescribeStream operation 
(need to use this to access shard list) at 10 transactions per second. Large 
parallel tasks calling this operation simultaneously for big Kinesis streams 
might cause issues, which will make it hard to decide on an appropriate 
backfire time & retry limit for the DescribeStream operation.
    
    Other than this concern, I think it will be absolutely fine to implement 
this only on the parallel tasks. And certainly the implementation will be much 
cleaner and friendly to future enhancements for Kinesis-side resharding, as you 
mentioned :)


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to