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

ASF GitHub Bot commented on FLINK-4582:
---------------------------------------

tweise commented on a change in pull request #6968: [FLINK-4582] [kinesis] 
Consuming data from DynamoDB streams to flink
URL: https://github.com/apache/flink/pull/6968#discussion_r236085419
 
 

 ##########
 File path: 
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java
 ##########
 @@ -0,0 +1,109 @@
+/*
+ * 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 org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import 
org.apache.flink.streaming.connectors.kinesis.internals.DynamodbStreamsDataFetcher;
+import 
org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher;
+import 
org.apache.flink.streaming.connectors.kinesis.serialization.DynamodbStreamsSchema;
+import 
org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema;
+
+import com.amazonaws.services.dynamodbv2.model.Record;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Consume events from the dyanmodbdb streams.
+ *
+ * @param <T> the type of data emitted
+ */
+public class FlinkDynamodbStreamsConsumer<T> extends FlinkKinesisConsumer<T> {
+       private static final Logger LOG = 
LoggerFactory.getLogger(FlinkDynamodbStreamsConsumer.class);
+
+       /**
+        * Constructor of FlinkDynamodbStreamsConsumer.
+        *
+        * @param stream stream to consume
+        * @param deserializer deserialization schema
+        * @param config config properties
+        */
+       public FlinkDynamodbStreamsConsumer(
+                       String stream,
+                       DeserializationSchema<T> deserializer,
+                       Properties config) {
+               super(stream, deserializer, config);
+       }
+
+       /**
+        * Constructor of FlinkDynamodbStreamConsumer.
+        *
+        * @param streams list of streams to consume
+        * @param deserializer  deserialization schema
+        * @param config config properties
+        */
+       public FlinkDynamodbStreamsConsumer(
+                       List<String> streams,
+                       KinesisDeserializationSchema deserializer,
+                       Properties config) {
+               super(streams, deserializer, config);
+       }
+
+       public static <T> FlinkDynamodbStreamsConsumer<T> create(String stream,
 
 Review comment:
   Please remove these static create methods, they don't provide any benefit 
since the constructors are public (which is consistent with the base class).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Allow FlinkKinesisConsumer to adapt for AWS DynamoDB Streams
> ------------------------------------------------------------
>
>                 Key: FLINK-4582
>                 URL: https://issues.apache.org/jira/browse/FLINK-4582
>             Project: Flink
>          Issue Type: New Feature
>          Components: Kinesis Connector, Streaming Connectors
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Ying Xu
>            Priority: Major
>              Labels: pull-request-available
>
> AWS DynamoDB is a NoSQL database service that has a CDC-like (change data 
> capture) feature called DynamoDB Streams 
> (http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Streams.html),
>  which is a stream feed of item-level table activities.
> The DynamoDB Streams shard abstraction follows that of Kinesis Streams with 
> only a slight difference in resharding behaviours, so it is possible to build 
> on the internals of our Flink Kinesis Consumer for an exactly-once DynamoDB 
> Streams source.
> I propose an API something like this:
> {code}
> DataStream dynamoItemsCdc = 
>   FlinkKinesisConsumer.asDynamoDBStream(tableNames, schema, config)
> {code}
> The feature adds more connectivity to popular AWS services for Flink, and 
> combining what Flink has for exactly-once semantics, out-of-core state 
> backends, and queryable state with CDC can have very strong use cases. For 
> this feature there should only be an extra dependency to the AWS Java SDK for 
> DynamoDB, which has Apache License 2.0.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to