+ * Dynamodb streams shard ID is a char string ranging from 28 characters to 65 characters. + * (See https://docs.aws.amazon.com/amazondynamodb/latest/APIReference/API_streams_Shard.html) + * + * The shardId observed usually takes the format of: "shardId-00000001536805703746-69688cb1", + * where "shardId-" is a prefix, followed by a 20-digit timestamp string and 0-36 or more + * characters, separated by '-'. Following this format, it is expected the child shards created + * during a re-sharding event have shardIds bigger than their parents. + *
+ * @param shardId shard Id + * @return boolean indicate if the given shard Id is valid + */ + public static boolean isValidShardId(String shardId) { + return shardId == null ? false : shardId.matches("^shardId-\\d{20}-{0,1}\\w{0,36}"); + } +} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java new file mode 100644 index 0000000000000..eb5620faf172a --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java @@ -0,0 +1,137 @@ +/* + * 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 org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.ClientConfigurationFactory; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.regions.Region; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.Shard; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_ENDPOINT; +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_REGION; +import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.getCredentialsProvider; +import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.setAwsClientConfigProperties; + +/** + * DynamoDB streams proxy: interface interacting with the DynamoDB streams. + */ +public class DynamoDBStreamsProxy extends KinesisProxy { + private static final Logger LOG = LoggerFactory.getLogger(DynamoDBStreamsProxy.class); + + /** Used for formatting Flink-specific user agent string when creating Kinesis client. */ + private static final String USER_AGENT_FORMAT = "Apache Flink %s (%s) DynamoDB Streams Connector"; + + protected DynamoDBStreamsProxy(Properties configProps) { + super(configProps); + } + + /** + * Creates a DynamoDB streams proxy. + * + * @param configProps configuration properties + * @return the created DynamoDB streams proxy + */ + public static KinesisProxyInterface create(Properties configProps) { + return new DynamoDBStreamsProxy(configProps); + } + + /** + * Creates an AmazonDynamoDBStreamsAdapterClient. + * Uses it as the internal client interacting with the DynamoDB streams. + * + * @param configProps configuration properties + * @return an AWS DynamoDB streams adapter client + */ + @Override + protected AmazonKinesis createKinesisClient(Properties configProps) { + ClientConfiguration awsClientConfig = new ClientConfigurationFactory().getConfig(); + setAwsClientConfigProperties(awsClientConfig, configProps); + + AWSCredentialsProvider credentials = getCredentialsProvider(configProps); + awsClientConfig.setUserAgentPrefix( + String.format( + USER_AGENT_FORMAT, + EnvironmentInformation.getVersion(), + EnvironmentInformation.getRevisionInformation().commitId)); + + AmazonDynamoDBStreamsAdapterClient adapterClient = + new AmazonDynamoDBStreamsAdapterClient(credentials, awsClientConfig); + + if (configProps.containsKey(AWS_ENDPOINT)) { + adapterClient.setEndpoint(configProps.getProperty(AWS_ENDPOINT)); + } else { + adapterClient.setRegion(Region.getRegion( + Regions.fromName(configProps.getProperty(AWS_REGION)))); + } + + return adapterClient; + } + + @Override + public GetShardListResult getShardList( + MapThis method is using a "full jitter" approach described in AWS's article,
+ *
+ * "Exponential Backoff and Jitter".
+ * This is necessary because concurrent calls will be made by all parallel subtask's fetcher.
+ * This jitter backoff approach will help distribute calls across the fetchers over time.
+ *
+ * @param streamName the stream to describe
+ * @param startShardId which shard to start with for this describe operation
+ *
+ * @return the result of the describe stream operation
+ */
+ protected DescribeStreamResult describeStream(String streamName, @Nullable String startShardId)
+ throws InterruptedException {
+ final DescribeStreamRequest describeStreamRequest = new DescribeStreamRequest();
+ describeStreamRequest.setStreamName(streamName);
+ describeStreamRequest.setExclusiveStartShardId(startShardId);
+
+ DescribeStreamResult describeStreamResult = null;
+
+ // Call DescribeStream, with full-jitter backoff (if we get LimitExceededException).
+ int attemptCount = 0;
+ while (describeStreamResult == null) { // retry until we get a result
+ try {
+ describeStreamResult = kinesisClient.describeStream(describeStreamRequest);
+ } catch (LimitExceededException le) {
+ long backoffMillis = fullJitterBackoff(
+ describeStreamBaseBackoffMillis,
+ describeStreamMaxBackoffMillis,
+ describeStreamExpConstant,
+ attemptCount++);
+ LOG.warn(String.format("Got LimitExceededException when describing stream %s. "
+ + "Backing off for %d millis.", streamName, backoffMillis));
+ Thread.sleep(backoffMillis);
+ } catch (ResourceNotFoundException re) {
+ throw new RuntimeException("Error while getting stream details", re);
+ }
+ }
+
+ String streamStatus = describeStreamResult.getStreamDescription().getStreamStatus();
+ if (!(streamStatus.equals(StreamStatus.ACTIVE.toString())
+ || streamStatus.equals(StreamStatus.UPDATING.toString()))) {
+ if (LOG.isWarnEnabled()) {
+ LOG.warn(String.format("The status of stream %s is %s ; result of the current "
+ + "describeStream operation will not contain any shard information.",
+ streamName, streamStatus));
+ }
+ }
+
+ return describeStreamResult;
+ }
+
protected static long fullJitterBackoff(long base, long max, double power, int attempt) {
long exponentialBackoff = (long) Math.min(max, base * Math.pow(power, attempt));
return (long) (seed.nextDouble() * exponentialBackoff); // random jitter between 0 and the exponential backoff
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java
new file mode 100644
index 0000000000000..ea6eeee917c47
--- /dev/null
+++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java
@@ -0,0 +1,44 @@
+/*
+ * 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.serialization;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import com.amazonaws.services.dynamodbv2.model.Record;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.IOException;
+
+/**
+ * Schema used for deserializing DynamoDB streams records.
+ */
+public class DynamoDBStreamsSchema implements KinesisDeserializationSchema
+ * A set of configuration paremeters associated with the describeStreams API may be used if:
+ * 1) an legacy client wants to consume from Kinesis
+ * 2) a current client wants to consumer from DynamoDB streams
+ *
+ * In the context of 1), the set of configurations needs to be translated to the corresponding
+ * configurations in the Kinesis listShards API. In the mean time, keep these configs since
+ * they are applicable in the context of 2), i.e., polling data from a DynamoDB stream.
+ *