flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-3229) Kinesis streaming consumer with integration of Flink's checkpointing mechanics
Date Fri, 22 Apr 2016 15:46:12 GMT

    [ https://issues.apache.org/jira/browse/FLINK-3229?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15254106#comment-15254106

ASF GitHub Bot commented on FLINK-3229:

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

    --- Diff: flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/KinesisStreamShard.java
    @@ -0,0 +1,160 @@
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.flink.streaming.connectors.kinesis.model;
    +import java.io.Serializable;
    +import static com.google.common.base.Preconditions.checkNotNull;
    + * A serializable representation of a AWS Kinesis Stream shard. It is basically a wrapper
class around the information
    + * provided along with {@link com.amazonaws.services.kinesis.model.Shard}, with some
extra utility methods to
    + * determine whether or not a shard is closed and whether or not the shard is a result
of parent shard splits or merges.
    + */
    +public class KinesisStreamShard implements Serializable {
    --- End diff --
    The main reason for why the consumer needs to have another Shard representation is because
`com.amazonaws.services.kinesis.model.Shard` doesn't have the shard's associated stream name
as a field. We will need the stream name when getting a shard iterator for a particular shard,
using `com.amazonaws.services.kinesis.AmazonKinesisClient#getShardIterator(streamName, shardId,
iteratorType)`. Moreover, since the consumer's implementation supports reading from multiple
Kinesis streams, we must carry the associated stream name along with each Shard representation
(I guess that's the reason why Amazon's Shard implementation doesn't have a field for stream
    Our implementation, `org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard`,
currently has `regionName` (I don't think we need this actually, since the consumer is limited
to read from Kinesis streams within the same region) and `streamName` as fields besides the
already supplied ones in Amazon's Shard. So, what we could do to reduce duplicate implementation
is to include Amazon's Shard implementation as a field within our `KinesisStreamShard`, and
let the `KinesisStreamShard` still have `streamName` as an extra field. How do you think?

> Kinesis streaming consumer with integration of Flink's checkpointing mechanics
> ------------------------------------------------------------------------------
>                 Key: FLINK-3229
>                 URL: https://issues.apache.org/jira/browse/FLINK-3229
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Streaming Connectors
>    Affects Versions: 1.0.0
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
> Opening a sub-task to implement data source consumer for Kinesis streaming connector
> An example of the planned user API for Flink Kinesis Consumer:
> {code}
> Properties kinesisConfig = new Properties();
> config.put(KinesisConfigConstants.CONFIG_AWS_REGION, "us-east-1");
> config.put(KinesisConfigConstants.CONFIG_AWS_CREDENTIALS_PROVIDER_TYPE, "BASIC");
> config.put(
>     "aws_access_key_id_here");
> config.put(
>     "aws_secret_key_here");
> config.put(KinesisConfigConstants.CONFIG_STREAM_INIT_POSITION_TYPE, "LATEST"); // or
> DataStream<T> kinesisRecords = env.addSource(new FlinkKinesisConsumer<>(
>     "kinesis_stream_name",
>     new SimpleStringSchema(),
>     kinesisConfig));
> {code}

This message was sent by Atlassian JIRA

View raw message