[
https://issues.apache.org/jira/browse/FLINK-4582?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16709203#comment-16709203
]
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_r238817945
##########
File path:
flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java
##########
@@ -143,6 +143,34 @@ public SentinelSequenceNumber toSentinelSequenceNumber() {
/** The interval after which to consider a shard idle for purposes of
watermark generation. */
public static final String SHARD_IDLE_INTERVAL_MILLIS =
"flink.shard.idle.interval";
+ /**
+ * The base backoff time between each describeStream attempt.
+ * Different tag name to distinguish from
"flink.stream.describe.backoff.base"
+ * since the latter is deprecated.
+ */
+ public static final String DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE =
Review comment:
It would be good to modify the method to map/replicate the keys but not
remove them (since you need them for dynamo). Add a comment that this is for
backward compatibility for the regular proxy that is now using listShards
instead of describeShards. We can remove the mapping in later release.
----------------------------------------------------------------
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)