From 22a2b4056a019d81708cbf9fb2cbbb8c6f82f483 Mon Sep 17 00:00:00 2001 From: Ying Date: Tue, 30 Oct 2018 13:07:50 -0700 Subject: [PATCH 01/15] FLINK-4582: initial commit for flink dynamodbstreams connector --- .../flink-connector-kinesis/pom.xml | 12 +- .../FlinkDynamodbStreamsConsumer.java | 108 +++++++++ .../config/ConsumerConfigConstants.java | 44 ++++ .../internals/DynamodbStreamsDataFetcher.java | 127 ++++++++++ .../model/DynamodbStreamsShardHandle.java | 68 ++++++ .../proxy/DynamodbStreamsProxy.java | 225 ++++++++++++++++++ .../serialization/DynamodbStreamsSchema.java | 45 ++++ .../kinesis/internals/KinesisDataFetcher.java | 2 +- .../kinesis/proxy/KinesisProxy.java | 2 +- .../examples/ConsumeFromDynamodbStreams.java | 43 ++++ .../model/DynamodbStreamsShardHandleTest.java | 105 ++++++++ 11 files changed, 777 insertions(+), 4 deletions(-) create mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java create mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java create mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java create mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java create mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/proxy/DynamodbStreamsProxy.java create mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java create mode 100644 flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java create mode 100644 flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 135f1dddcc943..63ad2ed45707e 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -36,6 +36,7 @@ under the License. 1.11.319 1.9.0 0.12.9 + 1.4.0 jar @@ -119,15 +120,22 @@ under the License. com.amazonaws - aws-java-sdk-dynamodb + aws-java-sdk-cloudwatch + + + + + com.amazonaws + dynamodb-streams-kinesis-adapter + ${aws.dynamodbstreams-kinesis-adapter.version} + com.amazonaws aws-java-sdk-cloudwatch - diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java new file mode 100644 index 0000000000000..f38f33a972699 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java @@ -0,0 +1,108 @@ +/* + * 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.dynamodbstreams; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.dynamodbstreams.internals.DynamodbStreamsDataFetcher; +import org.apache.flink.streaming.connectors.dynamodbstreams.serialization.DynamodbStreamsSchema; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; +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.List; +import java.util.Properties; + +/** + * Consume events from the dyanmodbdb streams. + * + * @param the type of data emitted + */ +public class FlinkDynamodbStreamsConsumer extends FlinkKinesisConsumer { + 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, + KinesisDeserializationSchema 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 streams, + KinesisDeserializationSchema deserializer, + Properties config) { + super(streams, deserializer, config); + } + + public static FlinkDynamodbStreamsConsumer create(String stream, + KinesisDeserializationSchema deserializer, + Properties config) { + return new FlinkDynamodbStreamsConsumer<>(stream, deserializer, config); + } + + public static FlinkDynamodbStreamsConsumer create(List streams, + KinesisDeserializationSchema deserializer, + Properties config) { + return new FlinkDynamodbStreamsConsumer<>(streams, deserializer, config); + } + + public static FlinkDynamodbStreamsConsumer create(String stream, + Properties config) { + return create(stream, new DynamodbStreamsSchema(), config); + } + + public static FlinkDynamodbStreamsConsumer create(List streams, + Properties config) { + return create(streams, new DynamodbStreamsSchema(), config); + } + + @Override + protected KinesisDataFetcher createFetcher( + List streams, + SourceFunction.SourceContext sourceContext, + RuntimeContext runtimeContext, + Properties configProps, + KinesisDeserializationSchema deserializationSchema) { + return new DynamodbStreamsDataFetcher( + streams, + sourceContext, + runtimeContext, + configProps, + deserializationSchema, + getShardAssigner()); + } +} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java new file mode 100644 index 0000000000000..7a94ed18a3c3a --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java @@ -0,0 +1,44 @@ +package org.apache.flink.streaming.connectors.dynamodbstreams.config; + +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; + +/** + * Optional consumer specific configuration keys and default values for {@link org.apache.flink.streaming.connectors.dynamodbstreams.FlinkDynamodbStreamsConsumer}. + */ +public class ConsumerConfigConstants extends AWSConfigConstants { + /** + * 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 STREAM_DESCRIBE_BACKOFF_BASE = + "flink.dynamodbstream.describe.backoff.base"; + + /** + * The maximum backoff time between each describeStream attempt. + * Different tag name to distinguish from "flink.stream.describe.backoff.max" + * since the latter is deprecated. + */ + public static final String STREAM_DESCRIBE_BACKOFF_MAX = + "flink.dynamodbstream.describe.backoff.max"; + + /** + * The power constant for exponential backoff between each describeStream attempt. + * Different tag name to distinguish from "flink.stream.describe.backoff.expcost" + * since the latter is deprecated. + */ + public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = + "flink.dynamodbstream.describe.backoff.expconst"; + + /** Boolean to imply whether to compare shards based on the Shard Handle format. */ + public static final String STREAM_SHARDID_FORMAT_CHECK = + "flink.dynamodbstream.shardid.format.check"; + + public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; + + public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L; + + public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; + + public static final String DEFAULT_STREAM_SHARDID_FORMAT_CHECK = "false"; +} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java new file mode 100644 index 0000000000000..c192390d70770 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java @@ -0,0 +1,127 @@ +/* + * 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.dynamodbstreams.internals; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.dynamodbstreams.model.DynamodbStreamsShardHandle; +import org.apache.flink.streaming.connectors.dynamodbstreams.proxy.DynamodbStreamsProxy; +import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner; +import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; +import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer; +import org.apache.flink.streaming.connectors.kinesis.metrics.ShardMetricsReporter; +import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; +import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; + +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_SHARDID_FORMAT_CHECK; +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_SHARDID_FORMAT_CHECK; + +/** + * Dynamodb streams data fetcher. + * @param type of fetched data. + */ +public class DynamodbStreamsDataFetcher extends KinesisDataFetcher { + private boolean shardIdFormatCheck = false; + + /** + * Constructor. + * + * @param streams list of streams to fetch data + * @param sourceContext source context + * @param runtimeContext runtime context + * @param configProps config properties + * @param deserializationSchema deserialization schema + * @param shardAssigner shard assigner + */ + public DynamodbStreamsDataFetcher(List streams, + SourceFunction.SourceContext sourceContext, + RuntimeContext runtimeContext, + Properties configProps, + KinesisDeserializationSchema deserializationSchema, + KinesisShardAssigner shardAssigner) { + + super(streams, + sourceContext, + sourceContext.getCheckpointLock(), + runtimeContext, + configProps, + deserializationSchema, + shardAssigner, + new AtomicReference<>(), + new ArrayList<>(), + createInitialSubscribedStreamsToLastDiscoveredShardsState(streams), + // use DynamodbStreamsProxy + DynamodbStreamsProxy::create); + + shardIdFormatCheck = Boolean.valueOf(configProps.getProperty( + STREAM_SHARDID_FORMAT_CHECK, + DEFAULT_STREAM_SHARDID_FORMAT_CHECK)); + } + + /** + * Updates the last discovered shard of a subscribed stream; only updates if the update is valid. + */ + @Override + public void advanceLastDiscoveredShardOfStream(String stream, String shardId) { + String lastSeenShardIdOfStream = subscribedStreamsToLastDiscoveredShardIds.get(stream); + + if (lastSeenShardIdOfStream == null) { + // if not previously set, simply put as the last seen shard id + subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); + } else { + if (shardIdFormatCheck && + DynamodbStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { + // Update is valid only if the given shard id is greater + // than the previous last seen shard id of the stream. + return; + } + subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); + } + } + + /** + * Create a new DynamoDB streams shard consumer. + * + * @param subscribedShardStateIndex the state index of the shard this consumer is subscribed to + * @param handle stream handle + * @param lastSeqNum last sequence number + * @param shardMetricsReporter the reporter to report metrics to + * @return + */ + @Override + protected ShardConsumer createShardConsumer( + Integer subscribedShardStateIndex, + StreamShardHandle handle, + SequenceNumber lastSeqNum, + ShardMetricsReporter shardMetricsReporter) { + + return new ShardConsumer( + this, + subscribedShardStateIndex, + handle, + lastSeqNum, + DynamodbStreamsProxy.create(getConsumerConfiguration()), + shardMetricsReporter); + } +} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java new file mode 100644 index 0000000000000..f52bba83168fd --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java @@ -0,0 +1,68 @@ +/* + * 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.dynamodbstreams.model; + +import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; + +import com.amazonaws.services.kinesis.model.Shard; + +/** + * DynamoDB streams shard handle format and utilities. + */ +public class DynamodbStreamsShardHandle extends StreamShardHandle{ + public static final String SHARDID_PREFIX = "shardId-"; + public static final int SHARDID_PREFIX_LEN = SHARDID_PREFIX.length(); + + public DynamodbStreamsShardHandle(String streamName, Shard shard) { + super(streamName, shard); + } + + public static int compareShardIds(String firstShardId, String secondShardId) { + if (!isValidShardId(firstShardId)) { + throw new IllegalArgumentException( + String.format("The first shard id %s has invalid format.", firstShardId)); + } else if (!isValidShardId(secondShardId)) { + throw new IllegalArgumentException( + String.format("The second shard id %s has invalid format.", secondShardId)); + } + + return firstShardId.substring(SHARDID_PREFIX_LEN).compareTo( + secondShardId.substring(SHARDID_PREFIX_LEN)); + } + + /** + *

+ * 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 shall take the form of: "shardId-00000001536805703746-69688cb1", + * where "shardId-" is a prefix, followed by a 20-digit string (timestamp) and 0-36 or more + * characters, separated by '-'. + * + * As long as the shardId conforms to this format, it is expected the new shards created + * during the re-sharding event has shardIds bigger than their parents. + * + * We would like to treat shardIds which do not conform to this format as invalid. + *

+ * @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/dynamodbstreams/proxy/DynamodbStreamsProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/proxy/DynamodbStreamsProxy.java new file mode 100644 index 0000000000000..1b77fd2ecb351 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/proxy/DynamodbStreamsProxy.java @@ -0,0 +1,225 @@ +/* + * 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.dynamodbstreams.proxy; + +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; +import org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; +import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; + +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.DescribeStreamRequest; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.ResourceNotFoundException; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.StreamStatus; +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.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX; +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX; +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"; + + // Backoff millis for the describe stream operation. + private final long describeStreamBaseBackoffMillis; + // Maximum backoff millis for the describe stream operation. + private final long describeStreamMaxBackoffMillis; + // Exponential backoff power constant for the describe stream operation. + private final double describeStreamExpConstant; + + protected DynamodbStreamsProxy(Properties configProps) { + super(configProps); + + // parse properties + describeStreamBaseBackoffMillis = Long.valueOf( + configProps.getProperty(STREAM_DESCRIBE_BACKOFF_BASE, + Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE))); + describeStreamMaxBackoffMillis = Long.valueOf( + configProps.getProperty(STREAM_DESCRIBE_BACKOFF_MAX, + Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX))); + describeStreamExpConstant = Double.valueOf( + configProps.getProperty(STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); + } + + + /** + * 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( + Map streamNamesWithLastSeenShardIds) throws InterruptedException { + GetShardListResult result = new GetShardListResult(); + + for (Map.Entry streamNameWithLastSeenShardId : + streamNamesWithLastSeenShardIds.entrySet()) { + String stream = streamNameWithLastSeenShardId.getKey(); + String lastSeenShardId = streamNameWithLastSeenShardId.getValue(); + result.addRetrievedShardsToStream(stream, getShardsOfStream(stream, lastSeenShardId)); + } + return result; + } + + private List getShardsOfStream( + String streamName, + @Nullable String lastSeenShardId) + throws InterruptedException { + List shardsOfStream = new ArrayList<>(); + + DescribeStreamResult describeStreamResult; + do { + describeStreamResult = describeStream(streamName, lastSeenShardId); + List shards = describeStreamResult.getStreamDescription().getShards(); + for (Shard shard : shards) { + shardsOfStream.add(new StreamShardHandle(streamName, shard)); + } + + if (shards.size() != 0) { + lastSeenShardId = shards.get(shards.size() - 1).getShardId(); + } + } while (describeStreamResult.getStreamDescription().isHasMoreShards()); + + return shardsOfStream; + } + + /** + * Get metainfo for a Dynamodb stream, which contains information about which shards this + * Dynamodb stream possess. + * + *

This 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 + */ + private 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; + } + +} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java new file mode 100644 index 0000000000000..e25f87694f2e6 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java @@ -0,0 +1,45 @@ +/* + * 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.dynamodbstreams.serialization; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; + +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 { + private static final ObjectMapper MAPPER = new ObjectMapper(); + + @Override + public Record deserialize(byte[] message, String partitionKey, String seqNum, + long approxArrivalTimestamp, String stream, String shardId) throws IOException { + return MAPPER.readValue(message, Record.class); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(Record.class); + } + +} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 77ca23c9d378e..72f64835f2d46 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -144,7 +144,7 @@ public class KinesisDataFetcher { /** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in. * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called. */ - private final Map subscribedStreamsToLastDiscoveredShardIds; + protected final Map subscribedStreamsToLastDiscoveredShardIds; /** * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index 262181ae3bcb2..1cf348ba6ca46 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -73,7 +73,7 @@ public class KinesisProxy implements KinesisProxyInterface { private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class); /** The actual Kinesis client from the AWS SDK that we will be using to make calls. */ - private final AmazonKinesis kinesisClient; + protected final AmazonKinesis kinesisClient; /** Random seed used to calculate backoff jitter for Kinesis operations. */ private static final Random seed = new Random(); diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java new file mode 100644 index 0000000000000..bbe03586a37b7 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java @@ -0,0 +1,43 @@ +package org.apache.flink.streaming.connectors.dynamodbstreams.examples; + +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; + +import java.util.Properties; + +/** + * Sample command-line program of consuming data from a single DynamoDB stream. + */ +public class ConsumeFromDynamodbStreams { + private static final String DYNAMODB_STREAM_NAME = "stream"; + + public static void main(String[] args) throws Exception { + ParameterTool pt = ParameterTool.fromArgs(args); + + StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); + see.setParallelism(1); + + Properties dynamodbStreamsConsumerConfig = new Properties(); + final String streamName = pt.getRequired(DYNAMODB_STREAM_NAME); + dynamodbStreamsConsumerConfig.setProperty( + ConsumerConfigConstants.AWS_REGION, pt.getRequired("region")); + dynamodbStreamsConsumerConfig.setProperty( + ConsumerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accesskey")); + dynamodbStreamsConsumerConfig.setProperty( + ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey")); + + DataStream dynamodbStreams = see.addSource(new FlinkKinesisConsumer<>( + streamName, + new SimpleStringSchema(), + dynamodbStreamsConsumerConfig)); + + dynamodbStreams.print(); + + see.execute(); + } + +} diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java new file mode 100644 index 0000000000000..720dc43bef98a --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java @@ -0,0 +1,105 @@ +/* + * 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.dynamodbstreams.model; + +import org.junit.Test; + +import static org.apache.flink.streaming.connectors.dynamodbstreams.model.DynamodbStreamsShardHandle.SHARDID_PREFIX; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Shard handle unit tests. + */ +public class DynamodbStreamsShardHandleTest { + @Test + public void testIsValidShardId() { + // normal form + String shardId = "shardId-00000001536805703746-69688cb1"; + assertEquals(true, DynamodbStreamsShardHandle.isValidShardId(shardId)); + + // short form + shardId = "shardId-00000001536805703746"; + assertEquals(true, DynamodbStreamsShardHandle.isValidShardId(shardId)); + + // long form + shardId = "shardId-00000001536805703746-69688cb1aljkwerijfl8228sl12a123akfla"; + assertEquals(true, DynamodbStreamsShardHandle.isValidShardId(shardId)); + + // invalid with wrong prefix + shardId = "sId-00000001536805703746-69688cb1"; + assertEquals(false, DynamodbStreamsShardHandle.isValidShardId(shardId)); + + // invalid with non-digits + shardId = "shardId-0000000153680570aabb-69688cb1"; + assertEquals(false, DynamodbStreamsShardHandle.isValidShardId(shardId)); + + // invalid with shardId too long + shardId = "shardId-00000001536805703746-69688cb1aljkwerijfl8228sl12a123akfla0000"; + assertEquals(false, DynamodbStreamsShardHandle.isValidShardId(shardId)); + } + + @Test + public void testCompareShardId() { + final int numShardIds = 10; + final int shardIdDigitLen = 20; + final String zeros = "00000000000000000000"; // twenty '0' chars + String shardIdValid = "shardId-00000001536805703746-69688cb1"; + String shardIdInvalid = "shardId-0000000153680570aabb-69688cb1"; + + assertEquals(0, DynamodbStreamsShardHandle.compareShardIds(shardIdValid, shardIdValid)); + + // comparison of invalid shardIds should yield exception + try { + DynamodbStreamsShardHandle.compareShardIds(shardIdValid, shardIdInvalid); + fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); + } catch (IllegalArgumentException e) { + // ignore + } + try { + DynamodbStreamsShardHandle.compareShardIds(shardIdInvalid, shardIdValid); + fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); + } catch (IllegalArgumentException e) { + // ignore + } + + // compare randomly generated shardIds based on timestamp + String[] shardIds = new String[numShardIds]; + for (int i = 0; i < numShardIds; i++) { + String nowStr = String.valueOf(System.currentTimeMillis()); + if (nowStr.length() < shardIdDigitLen) { + shardIds[i] = SHARDID_PREFIX + zeros.substring(0, shardIdDigitLen - nowStr.length()) + + nowStr; + } else { + shardIds[i] = SHARDID_PREFIX + nowStr.substring(0, shardIdDigitLen); + } + try { + Thread.sleep(100); + } catch (InterruptedException e) { + // ignore + } + } + for (int i = 1; i < numShardIds - 1; i++) { + assertTrue(DynamodbStreamsShardHandle.compareShardIds(shardIds[i - 1], shardIds[i]) < 0); + assertTrue(DynamodbStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i]) == 0); + assertTrue(DynamodbStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i + 1]) < 0); + } + } + +} From 03029cc2ea52c2f93b8c2579916163c5bba5aff0 Mon Sep 17 00:00:00 2001 From: Ying Date: Tue, 30 Oct 2018 15:45:48 -0700 Subject: [PATCH 02/15] FLINK-4582: small format fix --- .../config/ConsumerConfigConstants.java | 31 ++++++++++++++++--- .../model/DynamodbStreamsShardHandle.java | 12 +++---- .../examples/ConsumeFromDynamodbStreams.java | 17 ++++++++++ 3 files changed, 47 insertions(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java index 7a94ed18a3c3a..4138fc1043709 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java @@ -1,3 +1,20 @@ +/* + * 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.dynamodbstreams.config; import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; @@ -12,7 +29,7 @@ public class ConsumerConfigConstants extends AWSConfigConstants { * since the latter is deprecated. */ public static final String STREAM_DESCRIBE_BACKOFF_BASE = - "flink.dynamodbstream.describe.backoff.base"; + "flink.dynamodbstreams.describe.backoff.base"; /** * The maximum backoff time between each describeStream attempt. @@ -20,7 +37,7 @@ public class ConsumerConfigConstants extends AWSConfigConstants { * since the latter is deprecated. */ public static final String STREAM_DESCRIBE_BACKOFF_MAX = - "flink.dynamodbstream.describe.backoff.max"; + "flink.dynamodbstreams.describe.backoff.max"; /** * The power constant for exponential backoff between each describeStream attempt. @@ -28,11 +45,14 @@ public class ConsumerConfigConstants extends AWSConfigConstants { * since the latter is deprecated. */ public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = - "flink.dynamodbstream.describe.backoff.expconst"; + "flink.dynamodbstreams.describe.backoff.expconst"; - /** Boolean to imply whether to compare shards based on the Shard Handle format. */ + /** + * Boolean to indicate whether to compare/enforce shardId format based on the one defined in + * DynamodbStreamsShardHandle. + */ public static final String STREAM_SHARDID_FORMAT_CHECK = - "flink.dynamodbstream.shardid.format.check"; + "flink.dynamodbstreams.shardid.format.check"; public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; @@ -40,5 +60,6 @@ public class ConsumerConfigConstants extends AWSConfigConstants { public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; + // By default disable shardId format check. public static final String DEFAULT_STREAM_SHARDID_FORMAT_CHECK = "false"; } diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java index f52bba83168fd..ef1b1cc338b45 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java @@ -50,14 +50,10 @@ public static int compareShardIds(String firstShardId, String secondShardId) { * 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 shall take the form of: "shardId-00000001536805703746-69688cb1", - * where "shardId-" is a prefix, followed by a 20-digit string (timestamp) and 0-36 or more - * characters, separated by '-'. - * - * As long as the shardId conforms to this format, it is expected the new shards created - * during the re-sharding event has shardIds bigger than their parents. - * - * We would like to treat shardIds which do not conform to this format as invalid. + * 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 diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java index bbe03586a37b7..9723f4bc5db4e 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java @@ -1,3 +1,20 @@ +/* + * 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.dynamodbstreams.examples; import org.apache.flink.api.common.serialization.SimpleStringSchema; From a323e5529bcf19eebe20c92768abd84b3bb379cf Mon Sep 17 00:00:00 2001 From: Ying Date: Tue, 30 Oct 2018 16:55:08 -0700 Subject: [PATCH 03/15] Fix ConsumeFromDynamodbStreams example to use FlinkDynamodbStreamsConsumer --- .../dynamodbstreams/FlinkDynamodbStreamsConsumer.java | 8 +++++--- .../examples/ConsumeFromDynamodbStreams.java | 4 ++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java index f38f33a972699..ed90919f3b9a8 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.dynamodbstreams; 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.dynamodbstreams.internals.DynamodbStreamsDataFetcher; import org.apache.flink.streaming.connectors.dynamodbstreams.serialization.DynamodbStreamsSchema; @@ -29,6 +30,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collections; import java.util.List; import java.util.Properties; @@ -49,7 +51,7 @@ public class FlinkDynamodbStreamsConsumer extends FlinkKinesisConsumer { */ public FlinkDynamodbStreamsConsumer( String stream, - KinesisDeserializationSchema deserializer, + DeserializationSchema deserializer, Properties config) { super(stream, deserializer, config); } @@ -69,7 +71,7 @@ public FlinkDynamodbStreamsConsumer( } public static FlinkDynamodbStreamsConsumer create(String stream, - KinesisDeserializationSchema deserializer, + DeserializationSchema deserializer, Properties config) { return new FlinkDynamodbStreamsConsumer<>(stream, deserializer, config); } @@ -82,7 +84,7 @@ public static FlinkDynamodbStreamsConsumer create(List streams, public static FlinkDynamodbStreamsConsumer create(String stream, Properties config) { - return create(stream, new DynamodbStreamsSchema(), config); + return create(Collections.singletonList(stream), new DynamodbStreamsSchema(), config); } public static FlinkDynamodbStreamsConsumer create(List streams, diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java index 9723f4bc5db4e..4e7bbc4c52018 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java @@ -21,7 +21,7 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.dynamodbstreams.FlinkDynamodbStreamsConsumer; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import java.util.Properties; @@ -47,7 +47,7 @@ public static void main(String[] args) throws Exception { dynamodbStreamsConsumerConfig.setProperty( ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey")); - DataStream dynamodbStreams = see.addSource(new FlinkKinesisConsumer<>( + DataStream dynamodbStreams = see.addSource(new FlinkDynamodbStreamsConsumer<>( streamName, new SimpleStringSchema(), dynamodbStreamsConsumerConfig)); From 6c26109969c5efd2e5241f24892e272eb76eb68b Mon Sep 17 00:00:00 2001 From: Ying Date: Sun, 4 Nov 2018 21:13:06 -0800 Subject: [PATCH 04/15] address one review comment --- .../dynamodbstreams/model/DynamodbStreamsShardHandleTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java index 720dc43bef98a..ce0eacc957e4d 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java @@ -70,13 +70,13 @@ public void testCompareShardId() { DynamodbStreamsShardHandle.compareShardIds(shardIdValid, shardIdInvalid); fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); } catch (IllegalArgumentException e) { - // ignore + // expected } try { DynamodbStreamsShardHandle.compareShardIds(shardIdInvalid, shardIdValid); fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); } catch (IllegalArgumentException e) { - // ignore + // expected } // compare randomly generated shardIds based on timestamp From 06d7a5c89d5e12299ca1e230e8043bbff32cff53 Mon Sep 17 00:00:00 2001 From: Ying Date: Thu, 22 Nov 2018 00:04:50 -0800 Subject: [PATCH 05/15] FLINK-4582: adjust package name based on review comment --- .../config/ConsumerConfigConstants.java | 65 ------------------- .../FlinkDynamodbStreamsConsumer.java | 7 +- .../config/ConsumerConfigConstants.java | 37 +++++++++++ .../internals/DynamodbStreamsDataFetcher.java | 16 ++--- .../model/DynamodbStreamsShardHandle.java | 2 +- .../proxy/DynamodbStreamsProxy.java | 29 ++++----- .../serialization/DynamodbStreamsSchema.java | 2 +- .../examples/ConsumeFromDynamodbStreams.java | 4 +- .../model/DynamodbStreamsShardHandleTest.java | 5 +- 9 files changed, 67 insertions(+), 100 deletions(-) delete mode 100644 flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/FlinkDynamodbStreamsConsumer.java (91%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/internals/DynamodbStreamsDataFetcher.java (84%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/model/DynamodbStreamsShardHandle.java (97%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/proxy/DynamodbStreamsProxy.java (84%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/serialization/DynamodbStreamsSchema.java (95%) rename flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/examples/ConsumeFromDynamodbStreams.java (93%) rename flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/{dynamodbstreams => kinesis}/model/DynamodbStreamsShardHandleTest.java (93%) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java deleted file mode 100644 index 4138fc1043709..0000000000000 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/config/ConsumerConfigConstants.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.dynamodbstreams.config; - -import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; - -/** - * Optional consumer specific configuration keys and default values for {@link org.apache.flink.streaming.connectors.dynamodbstreams.FlinkDynamodbStreamsConsumer}. - */ -public class ConsumerConfigConstants extends AWSConfigConstants { - /** - * 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 STREAM_DESCRIBE_BACKOFF_BASE = - "flink.dynamodbstreams.describe.backoff.base"; - - /** - * The maximum backoff time between each describeStream attempt. - * Different tag name to distinguish from "flink.stream.describe.backoff.max" - * since the latter is deprecated. - */ - public static final String STREAM_DESCRIBE_BACKOFF_MAX = - "flink.dynamodbstreams.describe.backoff.max"; - - /** - * The power constant for exponential backoff between each describeStream attempt. - * Different tag name to distinguish from "flink.stream.describe.backoff.expcost" - * since the latter is deprecated. - */ - public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = - "flink.dynamodbstreams.describe.backoff.expconst"; - - /** - * Boolean to indicate whether to compare/enforce shardId format based on the one defined in - * DynamodbStreamsShardHandle. - */ - public static final String STREAM_SHARDID_FORMAT_CHECK = - "flink.dynamodbstreams.shardid.format.check"; - - public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; - - public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L; - - public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; - - // By default disable shardId format check. - public static final String DEFAULT_STREAM_SHARDID_FORMAT_CHECK = "false"; -} diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java similarity index 91% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java index ed90919f3b9a8..36eeeda63832e 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/FlinkDynamodbStreamsConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams; +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.dynamodbstreams.internals.DynamodbStreamsDataFetcher; -import org.apache.flink.streaming.connectors.dynamodbstreams.serialization.DynamodbStreamsSchema; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.internals.DynamodbStreamsDataFetcher; +import org.apache.flink.streaming.connectors.kinesis.serialization.DynamodbStreamsSchema; import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index 42e2173474b4c..cf174a8ae1139 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/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_STREAM_DESCRIBE_BACKOFF_BASE = + "flink.dynamodbstreams.describe.backoff.base"; + /** + * The maximum backoff time between each describeStream attempt. + * Different tag name to distinguish from "flink.stream.describe.backoff.max" + * since the latter is deprecated. + */ + public static final String DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX = + "flink.dynamodbstreams.describe.backoff.max"; + /** + * The power constant for exponential backoff between each describeStream attempt. + * Different tag name to distinguish from "flink.stream.describe.backoff.expcost" + * since the latter is deprecated. + */ + public static final String DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = + "flink.dynamodbstreams.describe.backoff.expconst"; + /** + * Boolean to indicate whether to compare/enforce shardId format based on the one defined in + * DynamodbStreamsShardHandle. + */ + public static final String DYNAMODB_STREAM_SHARDID_FORMAT_CHECK = + "flink.dynamodbstreams.shardid.format.check"; + // ------------------------------------------------------------------------ // Default values for consumer configuration // ------------------------------------------------------------------------ @@ -194,6 +222,15 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final long DEFAULT_SHARD_IDLE_INTERVAL_MILLIS = -1; + public static final long DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; + + public static final long DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX = 5000L; + + public static final double DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; + + // By default disable shardId format check. + public static final String DEFAULT_DYNAMODB_STREAM_SHARDID_FORMAT_CHECK = "false"; + /** * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured * getRecords interval can not exceed 5 minutes, which is the expire time for retrieved iterators. diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java similarity index 84% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java index c192390d70770..3e0e6a4febf02 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/internals/DynamodbStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams.internals; +package org.apache.flink.streaming.connectors.kinesis.internals; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.connectors.dynamodbstreams.model.DynamodbStreamsShardHandle; -import org.apache.flink.streaming.connectors.dynamodbstreams.proxy.DynamodbStreamsProxy; +import org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle; +import org.apache.flink.streaming.connectors.kinesis.proxy.DynamodbStreamsProxy; import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner; -import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; -import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer; import org.apache.flink.streaming.connectors.kinesis.metrics.ShardMetricsReporter; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; @@ -34,8 +32,8 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_SHARDID_FORMAT_CHECK; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_SHARDID_FORMAT_CHECK; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_SHARDID_FORMAT_CHECK; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_SHARDID_FORMAT_CHECK; /** * Dynamodb streams data fetcher. @@ -75,8 +73,8 @@ public DynamodbStreamsDataFetcher(List streams, DynamodbStreamsProxy::create); shardIdFormatCheck = Boolean.valueOf(configProps.getProperty( - STREAM_SHARDID_FORMAT_CHECK, - DEFAULT_STREAM_SHARDID_FORMAT_CHECK)); + DYNAMODB_STREAM_SHARDID_FORMAT_CHECK, + DEFAULT_DYNAMODB_STREAM_SHARDID_FORMAT_CHECK)); } /** diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java similarity index 97% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java index ef1b1cc338b45..da804f699dd8b 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandle.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams.model; +package org.apache.flink.streaming.connectors.kinesis.model; import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/proxy/DynamodbStreamsProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamodbStreamsProxy.java similarity index 84% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/proxy/DynamodbStreamsProxy.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamodbStreamsProxy.java index 1b77fd2ecb351..6761be4f7da40 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/proxy/DynamodbStreamsProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamodbStreamsProxy.java @@ -15,13 +15,10 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams.proxy; +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 org.apache.flink.streaming.connectors.kinesis.proxy.GetShardListResult; -import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxy; -import org.apache.flink.streaming.connectors.kinesis.proxy.KinesisProxyInterface; import com.amazonaws.ClientConfiguration; import com.amazonaws.ClientConfigurationFactory; @@ -46,12 +43,12 @@ import java.util.Map; import java.util.Properties; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.dynamodbstreams.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX; 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; @@ -78,14 +75,14 @@ protected DynamodbStreamsProxy(Properties configProps) { // parse properties describeStreamBaseBackoffMillis = Long.valueOf( - configProps.getProperty(STREAM_DESCRIBE_BACKOFF_BASE, - Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE))); + configProps.getProperty(DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE, + Long.toString(DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE))); describeStreamMaxBackoffMillis = Long.valueOf( - configProps.getProperty(STREAM_DESCRIBE_BACKOFF_MAX, - Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX))); + configProps.getProperty(DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX, + Long.toString(DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX))); describeStreamExpConstant = Double.valueOf( - configProps.getProperty(STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, - Double.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); + configProps.getProperty(DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); } diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamodbStreamsSchema.java similarity index 95% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamodbStreamsSchema.java index e25f87694f2e6..5f9c6c4f4e112 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/dynamodbstreams/serialization/DynamodbStreamsSchema.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamodbStreamsSchema.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams.serialization; +package org.apache.flink.streaming.connectors.kinesis.serialization; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamodbStreams.java similarity index 93% rename from flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java rename to flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamodbStreams.java index 4e7bbc4c52018..2a0eba3345c71 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/examples/ConsumeFromDynamodbStreams.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamodbStreams.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams.examples; +package org.apache.flink.streaming.connectors.kinesis.examples; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.dynamodbstreams.FlinkDynamodbStreamsConsumer; +import org.apache.flink.streaming.connectors.kinesis.FlinkDynamodbStreamsConsumer; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import java.util.Properties; diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java similarity index 93% rename from flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java rename to flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java index ce0eacc957e4d..a650b1b2bf0c5 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/dynamodbstreams/model/DynamodbStreamsShardHandleTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java @@ -15,11 +15,12 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.dynamodbstreams.model; +package org.apache.flink.streaming.connectors.kinesis.model; +import org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle; import org.junit.Test; -import static org.apache.flink.streaming.connectors.dynamodbstreams.model.DynamodbStreamsShardHandle.SHARDID_PREFIX; +import static org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle.SHARDID_PREFIX; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; From 43f5ec585ecc2d08bf977b3d3be240513d378a3d Mon Sep 17 00:00:00 2001 From: Ying Date: Thu, 22 Nov 2018 00:34:57 -0800 Subject: [PATCH 06/15] fix checkstyle --- .../connectors/kinesis/FlinkDynamodbStreamsConsumer.java | 2 +- .../kinesis/internals/DynamodbStreamsDataFetcher.java | 4 ++-- .../connectors/kinesis/model/DynamodbStreamsShardHandle.java | 2 -- .../connectors/kinesis/proxy/DynamodbStreamsProxy.java | 4 ++-- .../kinesis/serialization/DynamodbStreamsSchema.java | 1 - .../kinesis/model/DynamodbStreamsShardHandleTest.java | 1 - 6 files changed, 5 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java index 36eeeda63832e..c989e1699a525 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java @@ -21,8 +21,8 @@ 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.serialization.DynamodbStreamsSchema; 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; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java index 3e0e6a4febf02..68fe6d86783f6 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java @@ -19,12 +19,12 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle; -import org.apache.flink.streaming.connectors.kinesis.proxy.DynamodbStreamsProxy; import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner; import org.apache.flink.streaming.connectors.kinesis.metrics.ShardMetricsReporter; +import org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; +import org.apache.flink.streaming.connectors.kinesis.proxy.DynamodbStreamsProxy; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; import java.util.ArrayList; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java index da804f699dd8b..c8df2769bb467 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java @@ -17,8 +17,6 @@ package org.apache.flink.streaming.connectors.kinesis.model; -import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; - import com.amazonaws.services.kinesis.model.Shard; /** 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 index 6761be4f7da40..1bdfe6d11cd93 100644 --- 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 @@ -43,14 +43,14 @@ 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.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE; import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX; import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE; import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX; -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; 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 index 5f9c6c4f4e112..576f4da63c5a9 100644 --- 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 @@ -18,7 +18,6 @@ package org.apache.flink.streaming.connectors.kinesis.serialization; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; import com.amazonaws.services.dynamodbv2.model.Record; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java index a650b1b2bf0c5..579b2b46f0e0d 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java @@ -17,7 +17,6 @@ package org.apache.flink.streaming.connectors.kinesis.model; -import org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle; import org.junit.Test; import static org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle.SHARDID_PREFIX; From e6f300f3726686095561a34b52fd6369786a4695 Mon Sep 17 00:00:00 2001 From: Ying Date: Thu, 22 Nov 2018 11:41:38 -0800 Subject: [PATCH 07/15] minor naming adjustment --- .../config/ConsumerConfigConstants.java | 16 ++++++------- .../internals/DynamodbStreamsDataFetcher.java | 8 +++---- .../kinesis/proxy/DynamodbStreamsProxy.java | 24 +++++++++---------- 3 files changed, 24 insertions(+), 24 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index cf174a8ae1139..88c00aa02896e 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -148,27 +148,27 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { * Different tag name to distinguish from "flink.stream.describe.backoff.base" * since the latter is deprecated. */ - public static final String DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE = + public static final String DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE = "flink.dynamodbstreams.describe.backoff.base"; /** * The maximum backoff time between each describeStream attempt. * Different tag name to distinguish from "flink.stream.describe.backoff.max" * since the latter is deprecated. */ - public static final String DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX = + public static final String DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX = "flink.dynamodbstreams.describe.backoff.max"; /** * The power constant for exponential backoff between each describeStream attempt. * Different tag name to distinguish from "flink.stream.describe.backoff.expcost" * since the latter is deprecated. */ - public static final String DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = + public static final String DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = "flink.dynamodbstreams.describe.backoff.expconst"; /** * Boolean to indicate whether to compare/enforce shardId format based on the one defined in * DynamodbStreamsShardHandle. */ - public static final String DYNAMODB_STREAM_SHARDID_FORMAT_CHECK = + public static final String DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK = "flink.dynamodbstreams.shardid.format.check"; // ------------------------------------------------------------------------ @@ -222,14 +222,14 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final long DEFAULT_SHARD_IDLE_INTERVAL_MILLIS = -1; - public static final long DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; + public static final long DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE = 1000L; - public static final long DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX = 5000L; + public static final long DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX = 5000L; - public static final double DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; + public static final double DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; // By default disable shardId format check. - public static final String DEFAULT_DYNAMODB_STREAM_SHARDID_FORMAT_CHECK = "false"; + public static final String DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK = "false"; /** * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java index 68fe6d86783f6..75f0f11a6edff 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java @@ -32,8 +32,8 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_SHARDID_FORMAT_CHECK; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_SHARDID_FORMAT_CHECK; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK; /** * Dynamodb streams data fetcher. @@ -73,8 +73,8 @@ public DynamodbStreamsDataFetcher(List streams, DynamodbStreamsProxy::create); shardIdFormatCheck = Boolean.valueOf(configProps.getProperty( - DYNAMODB_STREAM_SHARDID_FORMAT_CHECK, - DEFAULT_DYNAMODB_STREAM_SHARDID_FORMAT_CHECK)); + DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK, + DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK)); } /** 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 index 1bdfe6d11cd93..9c5141662798c 100644 --- 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 @@ -45,12 +45,12 @@ 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.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX; import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.getCredentialsProvider; import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.setAwsClientConfigProperties; @@ -75,14 +75,14 @@ protected DynamodbStreamsProxy(Properties configProps) { // parse properties describeStreamBaseBackoffMillis = Long.valueOf( - configProps.getProperty(DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE, - Long.toString(DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_BASE))); + configProps.getProperty(DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE, + Long.toString(DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE))); describeStreamMaxBackoffMillis = Long.valueOf( - configProps.getProperty(DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX, - Long.toString(DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_MAX))); + configProps.getProperty(DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX, + Long.toString(DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX))); describeStreamExpConstant = Double.valueOf( - configProps.getProperty(DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, - Double.toString(DEFAULT_DYNAMODB_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); + configProps.getProperty(DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); } From 2f94fe71e41cda5504e6c1b30354aa2758178dea Mon Sep 17 00:00:00 2001 From: Ying Date: Thu, 22 Nov 2018 15:55:44 -0800 Subject: [PATCH 08/15] FLINK-4582: fix compile and checkstyle --- .../connectors/kinesis/internals/DynamodbStreamsDataFetcher.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java index 75f0f11a6edff..b7e829009bc01 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java @@ -66,6 +66,7 @@ public DynamodbStreamsDataFetcher(List streams, configProps, deserializationSchema, shardAssigner, + null, new AtomicReference<>(), new ArrayList<>(), createInitialSubscribedStreamsToLastDiscoveredShardsState(streams), From 073470b2a4525226666c1244bae4b398411aff9c Mon Sep 17 00:00:00 2001 From: Ying Date: Thu, 20 Dec 2018 17:38:30 -0800 Subject: [PATCH 09/15] FLINK-4582: move describeStream into KinesisProxy; other review commments --- .../kinesis/FlinkDynamodbStreamsConsumer.java | 25 ------ .../config/ConsumerConfigConstants.java | 55 ++---------- .../internals/DynamodbStreamsDataFetcher.java | 23 ++--- .../kinesis/internals/KinesisDataFetcher.java | 9 +- .../kinesis/proxy/DynamodbStreamsProxy.java | 84 ------------------ .../kinesis/proxy/KinesisProxy.java | 88 ++++++++++++++++++- .../kinesis/util/KinesisConfigUtil.java | 36 +++++--- 7 files changed, 129 insertions(+), 191 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java index c989e1699a525..4c7c242a3da11 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java @@ -22,14 +22,11 @@ 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; @@ -69,28 +66,6 @@ public FlinkDynamodbStreamsConsumer( super(streams, deserializer, config); } - public static FlinkDynamodbStreamsConsumer create(String stream, - DeserializationSchema deserializer, - Properties config) { - return new FlinkDynamodbStreamsConsumer<>(stream, deserializer, config); - } - - public static FlinkDynamodbStreamsConsumer create(List streams, - KinesisDeserializationSchema deserializer, - Properties config) { - return new FlinkDynamodbStreamsConsumer<>(streams, deserializer, config); - } - - public static FlinkDynamodbStreamsConsumer create(String stream, - Properties config) { - return create(Collections.singletonList(stream), new DynamodbStreamsSchema(), config); - } - - public static FlinkDynamodbStreamsConsumer create(List streams, - Properties config) { - return create(streams, new DynamodbStreamsSchema(), config); - } - @Override protected KinesisDataFetcher createFetcher( List streams, diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index 88c00aa02896e..65ab3b96eedd7 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -65,31 +65,16 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { /** The date format of initial timestamp to start reading Kinesis stream from (when AT_TIMESTAMP is set for STREAM_INITIAL_POSITION). */ public static final String STREAM_TIMESTAMP_DATE_FORMAT = "flink.stream.initpos.timestamp.format"; - /** - * Deprecated key. - * - * @deprecated Use {@link ConsumerConfigConstants#LIST_SHARDS_BACKOFF_BASE} instead - **/ - @Deprecated - /** The base backoff time between each describeStream attempt. */ + /** The base backoff time between each describeStream attempt (Used for consuming data from DynamoDB streams). */ public static final String STREAM_DESCRIBE_BACKOFF_BASE = "flink.stream.describe.backoff.base"; - /** - * Deprecated key. - * - * @deprecated Use {@link ConsumerConfigConstants#LIST_SHARDS_BACKOFF_MAX} instead - **/ - @Deprecated - /** The maximum backoff time between each describeStream attempt. */ + /** The maximum backoff time between each describeStream attempt (Used for consuming data from DynamoDB streams). */ public static final String STREAM_DESCRIBE_BACKOFF_MAX = "flink.stream.describe.backoff.max"; /** - * Deprecated key. - * - * @deprecated Use {@link ConsumerConfigConstants#LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT} instead - **/ - @Deprecated - /** The power constant for exponential backoff between each describeStream attempt. */ + * The power constant for exponential backoff between each describeStream attempt (Used for consuming data from + * DynamoDB streams). + */ public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = "flink.stream.describe.backoff.expconst"; /** The maximum number of listShards attempts if we get a recoverable exception. */ @@ -143,27 +128,6 @@ 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 = - "flink.dynamodbstreams.describe.backoff.base"; - /** - * The maximum backoff time between each describeStream attempt. - * Different tag name to distinguish from "flink.stream.describe.backoff.max" - * since the latter is deprecated. - */ - public static final String DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX = - "flink.dynamodbstreams.describe.backoff.max"; - /** - * The power constant for exponential backoff between each describeStream attempt. - * Different tag name to distinguish from "flink.stream.describe.backoff.expcost" - * since the latter is deprecated. - */ - public static final String DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = - "flink.dynamodbstreams.describe.backoff.expconst"; /** * Boolean to indicate whether to compare/enforce shardId format based on the one defined in * DynamodbStreamsShardHandle. @@ -179,13 +143,10 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final String DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"; - @Deprecated public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE = 1000L; - @Deprecated public static final long DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX = 5000L; - @Deprecated public static final double DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; public static final long DEFAULT_LIST_SHARDS_BACKOFF_BASE = 1000L; @@ -222,12 +183,6 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final long DEFAULT_SHARD_IDLE_INTERVAL_MILLIS = -1; - public static final long DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE = 1000L; - - public static final long DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX = 5000L; - - public static final double DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = 1.5; - // By default disable shardId format check. public static final String DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK = "false"; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java index b7e829009bc01..a61f1acba516f 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java @@ -78,25 +78,16 @@ public DynamodbStreamsDataFetcher(List streams, DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK)); } - /** - * Updates the last discovered shard of a subscribed stream; only updates if the update is valid. - */ @Override - public void advanceLastDiscoveredShardOfStream(String stream, String shardId) { - String lastSeenShardIdOfStream = subscribedStreamsToLastDiscoveredShardIds.get(stream); - - if (lastSeenShardIdOfStream == null) { - // if not previously set, simply put as the last seen shard id - subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); - } else { - if (shardIdFormatCheck && + protected boolean shouldAdvanceLastDiscoveredShardId(String shardId, String lastSeenShardIdOfStream) { + if (shardIdFormatCheck && DynamodbStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { - // Update is valid only if the given shard id is greater - // than the previous last seen shard id of the stream. - return; - } - subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); + // shardID update is valid only if the given shard id is greater + // than the previous last seen shard id of the stream. + return false; } + + return true; } /** diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java index 72f64835f2d46..9cebc41ad2606 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/KinesisDataFetcher.java @@ -144,7 +144,7 @@ public class KinesisDataFetcher { /** The last discovered shard ids of each subscribed stream, updated as the fetcher discovers new shards in. * Note: this state will be updated if new shards are found when {@link KinesisDataFetcher#discoverNewShardsToSubscribe()} is called. */ - protected final Map subscribedStreamsToLastDiscoveredShardIds; + private final Map subscribedStreamsToLastDiscoveredShardIds; /** * The shards, along with their last processed sequence numbers, that this fetcher is subscribed to. The fetcher @@ -518,11 +518,16 @@ public void advanceLastDiscoveredShardOfStream(String stream, String shardId) { if (lastSeenShardIdOfStream == null) { // if not previously set, simply put as the last seen shard id this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); - } else if (StreamShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) > 0) { + } else if (shouldAdvanceLastDiscoveredShardId(shardId, lastSeenShardIdOfStream)) { this.subscribedStreamsToLastDiscoveredShardIds.put(stream, shardId); } } + /** Given lastSeenShardId, check if last discovered shardId should be advanced. */ + protected boolean shouldAdvanceLastDiscoveredShardId(String shardId, String lastSeenShardIdOfStream) { + return (StreamShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) > 0); + } + /** * A utility function that does the following: * 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 index 9c5141662798c..70bd9c9a3a287 100644 --- 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 @@ -27,12 +27,8 @@ import com.amazonaws.regions.Regions; import com.amazonaws.services.dynamodbv2.streamsadapter.AmazonDynamoDBStreamsAdapterClient; import com.amazonaws.services.kinesis.AmazonKinesis; -import com.amazonaws.services.kinesis.model.DescribeStreamRequest; import com.amazonaws.services.kinesis.model.DescribeStreamResult; -import com.amazonaws.services.kinesis.model.LimitExceededException; -import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.Shard; -import com.amazonaws.services.kinesis.model.StreamStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,12 +41,6 @@ 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.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX; import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.getCredentialsProvider; import static org.apache.flink.streaming.connectors.kinesis.util.AWSUtil.setAwsClientConfigProperties; @@ -63,26 +53,8 @@ public class DynamodbStreamsProxy extends KinesisProxy { /** 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"; - // Backoff millis for the describe stream operation. - private final long describeStreamBaseBackoffMillis; - // Maximum backoff millis for the describe stream operation. - private final long describeStreamMaxBackoffMillis; - // Exponential backoff power constant for the describe stream operation. - private final double describeStreamExpConstant; - protected DynamodbStreamsProxy(Properties configProps) { super(configProps); - - // parse properties - describeStreamBaseBackoffMillis = Long.valueOf( - configProps.getProperty(DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE, - Long.toString(DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_BASE))); - describeStreamMaxBackoffMillis = Long.valueOf( - configProps.getProperty(DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX, - Long.toString(DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_MAX))); - describeStreamExpConstant = Double.valueOf( - configProps.getProperty(DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, - Double.toString(DEFAULT_DYNAMODB_STREAMS_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); } @@ -163,60 +135,4 @@ private List getShardsOfStream( return shardsOfStream; } - - /** - * Get metainfo for a Dynamodb stream, which contains information about which shards this - * Dynamodb stream possess. - * - *

This 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 - */ - private 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; - } - } diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index 1cf348ba6ca46..a0365038e6ad5 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -28,6 +28,8 @@ import com.amazonaws.ClientConfigurationFactory; import com.amazonaws.SdkClientException; import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.DescribeStreamRequest; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; import com.amazonaws.services.kinesis.model.ExpiredNextTokenException; import com.amazonaws.services.kinesis.model.GetRecordsRequest; import com.amazonaws.services.kinesis.model.GetRecordsResult; @@ -42,6 +44,7 @@ import com.amazonaws.services.kinesis.model.ResourceNotFoundException; import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.StreamStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,6 +58,12 @@ import java.util.Properties; import java.util.Random; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -73,7 +82,7 @@ public class KinesisProxy implements KinesisProxyInterface { private static final Logger LOG = LoggerFactory.getLogger(KinesisProxy.class); /** The actual Kinesis client from the AWS SDK that we will be using to make calls. */ - protected final AmazonKinesis kinesisClient; + private final AmazonKinesis kinesisClient; /** Random seed used to calculate backoff jitter for Kinesis operations. */ private static final Random seed = new Random(); @@ -126,6 +135,16 @@ public class KinesisProxy implements KinesisProxyInterface { /** Maximum retry attempts for the get shard iterator operation. */ private final int getShardIteratorMaxRetries; + + /* Backoff millis for the describe stream operation. */ + private final long describeStreamBaseBackoffMillis; + + /* Maximum backoff millis for the describe stream operation. */ + private final long describeStreamMaxBackoffMillis; + + /* Exponential backoff power constant for the describe stream operation. */ + private final double describeStreamExpConstant; + /** * Create a new KinesisProxy based on the supplied configuration properties. * @@ -133,7 +152,7 @@ public class KinesisProxy implements KinesisProxyInterface { */ protected KinesisProxy(Properties configProps) { checkNotNull(configProps); - KinesisConfigUtil.replaceDeprecatedConsumerKeys(configProps); + KinesisConfigUtil.backfillConsumerKeys(configProps); this.kinesisClient = createKinesisClient(configProps); @@ -153,7 +172,15 @@ protected KinesisProxy(Properties configProps) { configProps.getProperty( ConsumerConfigConstants.LIST_SHARDS_RETRIES, Long.toString(ConsumerConfigConstants.DEFAULT_LIST_SHARDS_RETRIES))); - + this.describeStreamBaseBackoffMillis = Long.valueOf( + configProps.getProperty(STREAM_DESCRIBE_BACKOFF_BASE, + Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE))); + this.describeStreamMaxBackoffMillis = Long.valueOf( + configProps.getProperty(STREAM_DESCRIBE_BACKOFF_MAX, + Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX))); + this.describeStreamExpConstant = Double.valueOf( + configProps.getProperty(STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); this.getRecordsBaseBackoffMillis = Long.valueOf( configProps.getProperty( ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE, @@ -469,6 +496,61 @@ private ListShardsResult listShards(String streamName, @Nullable String startSha return listShardsResults; } + /** + * Get metainfo for a Dynamodb stream, which contains information about which shards this + * Dynamodb stream possess. + * + *

This 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/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java index 75c84cdca5a27..45554a5b12541 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java @@ -193,18 +193,32 @@ public static Properties replaceDeprecatedProducerKeys(Properties configProps) { return configProps; } - public static Properties replaceDeprecatedConsumerKeys(Properties configProps) { - HashMap deprecatedOldKeyToNewKeys = new HashMap<>(); - deprecatedOldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_BASE); - deprecatedOldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_MAX); - deprecatedOldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT); - for (Map.Entry entry : deprecatedOldKeyToNewKeys.entrySet()) { - String deprecatedOldKey = entry.getKey(); + /** + *

+ * 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 may be useful in the context of 2), i.e., polling data from a DynamoDB stream. + *

+ * + * @param configProps original config properties. + * @return backfilled config properties. + */ + public static Properties backfillConsumerKeys(Properties configProps) { + HashMap oldKeyToNewKeys = new HashMap<>(); + oldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_BASE); + oldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_MAX); + oldKeyToNewKeys.put(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, ConsumerConfigConstants.LIST_SHARDS_BACKOFF_EXPONENTIAL_CONSTANT); + for (Map.Entry entry : oldKeyToNewKeys.entrySet()) { + String oldKey = entry.getKey(); String newKey = entry.getValue(); - if (configProps.containsKey(deprecatedOldKey)) { - LOG.warn("Please note {} property has been deprecated. Please use the {} new property key", deprecatedOldKey, newKey); - configProps.setProperty(newKey, configProps.getProperty(deprecatedOldKey)); - configProps.remove(deprecatedOldKey); + if (configProps.containsKey(oldKey)) { + LOG.warn("Backfill the property key {} based on the original key {}", newKey, oldKey); + configProps.setProperty(newKey, configProps.getProperty(oldKey)); + // Do not remove the oldKey since they may be used in the context of talking to DynamoDB streams } } return configProps; From c08e9b2bd6f8e1dba5f078efac06ed1516150463 Mon Sep 17 00:00:00 2001 From: Ying Date: Fri, 21 Dec 2018 00:23:29 -0800 Subject: [PATCH 10/15] FLINK-4582: more review comments --- ...java => FlinkDynamoDBStreamsConsumer.java} | 14 +++++----- .../config/ConsumerConfigConstants.java | 2 +- ...r.java => DynamoDBStreamsDataFetcher.java} | 24 ++++++++-------- ...e.java => DynamoDBStreamsShardHandle.java} | 4 +-- ...msProxy.java => DynamoDBStreamsProxy.java} | 9 +++--- .../kinesis/proxy/KinesisProxy.java | 22 ++++++--------- ...Schema.java => DynamoDBStreamsSchema.java} | 2 +- .../kinesis/util/KinesisConfigUtil.java | 3 +- ...s.java => ConsumeFromDynamoDBStreams.java} | 6 ++-- ...va => DynamoDBStreamsShardHandleTest.java} | 28 +++++++++---------- 10 files changed, 52 insertions(+), 62 deletions(-) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/{FlinkDynamodbStreamsConsumer.java => FlinkDynamoDBStreamsConsumer.java} (89%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/{DynamodbStreamsDataFetcher.java => DynamoDBStreamsDataFetcher.java} (83%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/{DynamodbStreamsShardHandle.java => DynamoDBStreamsShardHandle.java} (95%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/{DynamodbStreamsProxy.java => DynamoDBStreamsProxy.java} (96%) rename flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/{DynamodbStreamsSchema.java => DynamoDBStreamsSchema.java} (96%) rename flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/{ConsumeFromDynamodbStreams.java => ConsumeFromDynamoDBStreams.java} (96%) rename flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/{DynamodbStreamsShardHandleTest.java => DynamoDBStreamsShardHandleTest.java} (78%) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java similarity index 89% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java index 4c7c242a3da11..c1395bd794565 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamodbStreamsConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java @@ -20,7 +20,7 @@ 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.DynamoDBStreamsDataFetcher; import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; @@ -35,17 +35,17 @@ * * @param the type of data emitted */ -public class FlinkDynamodbStreamsConsumer extends FlinkKinesisConsumer { - private static final Logger LOG = LoggerFactory.getLogger(FlinkDynamodbStreamsConsumer.class); +public class FlinkDynamoDBStreamsConsumer extends FlinkKinesisConsumer { + private static final Logger LOG = LoggerFactory.getLogger(FlinkDynamoDBStreamsConsumer.class); /** - * Constructor of FlinkDynamodbStreamsConsumer. + * Constructor of FlinkDynamoDBStreamsConsumer. * * @param stream stream to consume * @param deserializer deserialization schema * @param config config properties */ - public FlinkDynamodbStreamsConsumer( + public FlinkDynamoDBStreamsConsumer( String stream, DeserializationSchema deserializer, Properties config) { @@ -59,7 +59,7 @@ public FlinkDynamodbStreamsConsumer( * @param deserializer deserialization schema * @param config config properties */ - public FlinkDynamodbStreamsConsumer( + public FlinkDynamoDBStreamsConsumer( List streams, KinesisDeserializationSchema deserializer, Properties config) { @@ -73,7 +73,7 @@ protected KinesisDataFetcher createFetcher( RuntimeContext runtimeContext, Properties configProps, KinesisDeserializationSchema deserializationSchema) { - return new DynamodbStreamsDataFetcher( + return new DynamoDBStreamsDataFetcher( streams, sourceContext, runtimeContext, diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index 65ab3b96eedd7..a0449e0a57775 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -130,7 +130,7 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { /** * Boolean to indicate whether to compare/enforce shardId format based on the one defined in - * DynamodbStreamsShardHandle. + * DynamoDBStreamsShardHandle. */ public static final String DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK = "flink.dynamodbstreams.shardid.format.check"; diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java similarity index 83% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java index a61f1acba516f..b095fb2a4c37c 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamodbStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java @@ -20,11 +20,12 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.metrics.ShardMetricsReporter; -import org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle; +import org.apache.flink.streaming.connectors.kinesis.model.DynamoDBStreamsShardHandle; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; import org.apache.flink.streaming.connectors.kinesis.model.StreamShardHandle; -import org.apache.flink.streaming.connectors.kinesis.proxy.DynamodbStreamsProxy; +import org.apache.flink.streaming.connectors.kinesis.proxy.DynamoDBStreamsProxy; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchema; import java.util.ArrayList; @@ -32,14 +33,11 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK; - /** * Dynamodb streams data fetcher. * @param type of fetched data. */ -public class DynamodbStreamsDataFetcher extends KinesisDataFetcher { +public class DynamoDBStreamsDataFetcher extends KinesisDataFetcher { private boolean shardIdFormatCheck = false; /** @@ -52,7 +50,7 @@ public class DynamodbStreamsDataFetcher extends KinesisDataFetcher { * @param deserializationSchema deserialization schema * @param shardAssigner shard assigner */ - public DynamodbStreamsDataFetcher(List streams, + public DynamoDBStreamsDataFetcher(List streams, SourceFunction.SourceContext sourceContext, RuntimeContext runtimeContext, Properties configProps, @@ -70,18 +68,18 @@ public DynamodbStreamsDataFetcher(List streams, new AtomicReference<>(), new ArrayList<>(), createInitialSubscribedStreamsToLastDiscoveredShardsState(streams), - // use DynamodbStreamsProxy - DynamodbStreamsProxy::create); + // use DynamoDBStreamsProxy + DynamoDBStreamsProxy::create); shardIdFormatCheck = Boolean.valueOf(configProps.getProperty( - DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK, - DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK)); + ConsumerConfigConstants.DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK, + ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK)); } @Override protected boolean shouldAdvanceLastDiscoveredShardId(String shardId, String lastSeenShardIdOfStream) { if (shardIdFormatCheck && - DynamodbStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { + DynamoDBStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { // shardID update is valid only if the given shard id is greater // than the previous last seen shard id of the stream. return false; @@ -111,7 +109,7 @@ protected ShardConsumer createShardConsumer( subscribedShardStateIndex, handle, lastSeqNum, - DynamodbStreamsProxy.create(getConsumerConfiguration()), + DynamoDBStreamsProxy.create(getConsumerConfiguration()), shardMetricsReporter); } } diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandle.java similarity index 95% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandle.java index c8df2769bb467..534184f4ad812 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandle.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandle.java @@ -22,11 +22,11 @@ /** * DynamoDB streams shard handle format and utilities. */ -public class DynamodbStreamsShardHandle extends StreamShardHandle{ +public class DynamoDBStreamsShardHandle extends StreamShardHandle{ public static final String SHARDID_PREFIX = "shardId-"; public static final int SHARDID_PREFIX_LEN = SHARDID_PREFIX.length(); - public DynamodbStreamsShardHandle(String streamName, Shard shard) { + public DynamoDBStreamsShardHandle(String streamName, Shard shard) { super(streamName, shard); } 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 similarity index 96% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamodbStreamsProxy.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/DynamoDBStreamsProxy.java index 70bd9c9a3a287..eb5620faf172a 100644 --- 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 @@ -47,17 +47,16 @@ /** * DynamoDB streams proxy: interface interacting with the DynamoDB streams. */ -public class DynamodbStreamsProxy extends KinesisProxy { - private static final Logger LOG = LoggerFactory.getLogger(DynamodbStreamsProxy.class); +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) { + protected DynamoDBStreamsProxy(Properties configProps) { super(configProps); } - /** * Creates a DynamoDB streams proxy. * @@ -65,7 +64,7 @@ protected DynamodbStreamsProxy(Properties configProps) { * @return the created DynamoDB streams proxy */ public static KinesisProxyInterface create(Properties configProps) { - return new DynamodbStreamsProxy(configProps); + return new DynamoDBStreamsProxy(configProps); } /** diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index a0365038e6ad5..10a4774345b46 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -58,12 +58,6 @@ import java.util.Properties; import java.util.Random; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT; -import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -173,14 +167,14 @@ protected KinesisProxy(Properties configProps) { ConsumerConfigConstants.LIST_SHARDS_RETRIES, Long.toString(ConsumerConfigConstants.DEFAULT_LIST_SHARDS_RETRIES))); this.describeStreamBaseBackoffMillis = Long.valueOf( - configProps.getProperty(STREAM_DESCRIBE_BACKOFF_BASE, - Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE))); + configProps.getProperty(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_BASE, + Long.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_BASE))); this.describeStreamMaxBackoffMillis = Long.valueOf( - configProps.getProperty(STREAM_DESCRIBE_BACKOFF_MAX, - Long.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX))); + configProps.getProperty(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_MAX, + Long.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_MAX))); this.describeStreamExpConstant = Double.valueOf( - configProps.getProperty(STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, - Double.toString(DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); + configProps.getProperty(ConsumerConfigConstants.STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT, + Double.toString(ConsumerConfigConstants.DEFAULT_STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT))); this.getRecordsBaseBackoffMillis = Long.valueOf( configProps.getProperty( ConsumerConfigConstants.SHARD_GETRECORDS_BACKOFF_BASE, @@ -497,8 +491,8 @@ private ListShardsResult listShards(String streamName, @Nullable String startSha } /** - * Get metainfo for a Dynamodb stream, which contains information about which shards this - * Dynamodb stream possess. + * Get metainfo for a Kinesis stream, which contains information about which shards this + * Kinesis stream possess. * *

This method is using a "full jitter" approach described in AWS's article, * 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 similarity index 96% rename from flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamodbStreamsSchema.java rename to flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/serialization/DynamoDBStreamsSchema.java index 576f4da63c5a9..ea6eeee917c47 100644 --- 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 @@ -27,7 +27,7 @@ /** * Schema used for deserializing DynamoDB streams records. */ -public class DynamodbStreamsSchema implements KinesisDeserializationSchema { +public class DynamoDBStreamsSchema implements KinesisDeserializationSchema { private static final ObjectMapper MAPPER = new ObjectMapper(); @Override diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java index 45554a5b12541..b277696bf1a51 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java @@ -201,7 +201,7 @@ public static Properties replaceDeprecatedProducerKeys(Properties configProps) { * * 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 may be useful in the context of 2), i.e., polling data from a DynamoDB stream. + * they are applicable in the context of 2), i.e., polling data from a DynamoDB stream. *

* * @param configProps original config properties. @@ -216,7 +216,6 @@ public static Properties backfillConsumerKeys(Properties configProps) { String oldKey = entry.getKey(); String newKey = entry.getValue(); if (configProps.containsKey(oldKey)) { - LOG.warn("Backfill the property key {} based on the original key {}", newKey, oldKey); configProps.setProperty(newKey, configProps.getProperty(oldKey)); // Do not remove the oldKey since they may be used in the context of talking to DynamoDB streams } diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamodbStreams.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamoDBStreams.java similarity index 96% rename from flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamodbStreams.java rename to flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamoDBStreams.java index 2a0eba3345c71..44b1ddd1f441e 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamodbStreams.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/examples/ConsumeFromDynamoDBStreams.java @@ -21,7 +21,7 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.kinesis.FlinkDynamodbStreamsConsumer; +import org.apache.flink.streaming.connectors.kinesis.FlinkDynamoDBStreamsConsumer; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import java.util.Properties; @@ -29,7 +29,7 @@ /** * Sample command-line program of consuming data from a single DynamoDB stream. */ -public class ConsumeFromDynamodbStreams { +public class ConsumeFromDynamoDBStreams { private static final String DYNAMODB_STREAM_NAME = "stream"; public static void main(String[] args) throws Exception { @@ -47,7 +47,7 @@ public static void main(String[] args) throws Exception { dynamodbStreamsConsumerConfig.setProperty( ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey")); - DataStream dynamodbStreams = see.addSource(new FlinkDynamodbStreamsConsumer<>( + DataStream dynamodbStreams = see.addSource(new FlinkDynamoDBStreamsConsumer<>( streamName, new SimpleStringSchema(), dynamodbStreamsConsumerConfig)); diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandleTest.java similarity index 78% rename from flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java rename to flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandleTest.java index 579b2b46f0e0d..c8e3415315661 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamodbStreamsShardHandleTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/model/DynamoDBStreamsShardHandleTest.java @@ -19,7 +19,7 @@ import org.junit.Test; -import static org.apache.flink.streaming.connectors.kinesis.model.DynamodbStreamsShardHandle.SHARDID_PREFIX; +import static org.apache.flink.streaming.connectors.kinesis.model.DynamoDBStreamsShardHandle.SHARDID_PREFIX; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -27,32 +27,32 @@ /** * Shard handle unit tests. */ -public class DynamodbStreamsShardHandleTest { +public class DynamoDBStreamsShardHandleTest { @Test public void testIsValidShardId() { // normal form String shardId = "shardId-00000001536805703746-69688cb1"; - assertEquals(true, DynamodbStreamsShardHandle.isValidShardId(shardId)); + assertEquals(true, DynamoDBStreamsShardHandle.isValidShardId(shardId)); // short form shardId = "shardId-00000001536805703746"; - assertEquals(true, DynamodbStreamsShardHandle.isValidShardId(shardId)); + assertEquals(true, DynamoDBStreamsShardHandle.isValidShardId(shardId)); // long form shardId = "shardId-00000001536805703746-69688cb1aljkwerijfl8228sl12a123akfla"; - assertEquals(true, DynamodbStreamsShardHandle.isValidShardId(shardId)); + assertEquals(true, DynamoDBStreamsShardHandle.isValidShardId(shardId)); // invalid with wrong prefix shardId = "sId-00000001536805703746-69688cb1"; - assertEquals(false, DynamodbStreamsShardHandle.isValidShardId(shardId)); + assertEquals(false, DynamoDBStreamsShardHandle.isValidShardId(shardId)); // invalid with non-digits shardId = "shardId-0000000153680570aabb-69688cb1"; - assertEquals(false, DynamodbStreamsShardHandle.isValidShardId(shardId)); + assertEquals(false, DynamoDBStreamsShardHandle.isValidShardId(shardId)); // invalid with shardId too long shardId = "shardId-00000001536805703746-69688cb1aljkwerijfl8228sl12a123akfla0000"; - assertEquals(false, DynamodbStreamsShardHandle.isValidShardId(shardId)); + assertEquals(false, DynamoDBStreamsShardHandle.isValidShardId(shardId)); } @Test @@ -63,17 +63,17 @@ public void testCompareShardId() { String shardIdValid = "shardId-00000001536805703746-69688cb1"; String shardIdInvalid = "shardId-0000000153680570aabb-69688cb1"; - assertEquals(0, DynamodbStreamsShardHandle.compareShardIds(shardIdValid, shardIdValid)); + assertEquals(0, DynamoDBStreamsShardHandle.compareShardIds(shardIdValid, shardIdValid)); // comparison of invalid shardIds should yield exception try { - DynamodbStreamsShardHandle.compareShardIds(shardIdValid, shardIdInvalid); + DynamoDBStreamsShardHandle.compareShardIds(shardIdValid, shardIdInvalid); fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); } catch (IllegalArgumentException e) { // expected } try { - DynamodbStreamsShardHandle.compareShardIds(shardIdInvalid, shardIdValid); + DynamoDBStreamsShardHandle.compareShardIds(shardIdInvalid, shardIdValid); fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); } catch (IllegalArgumentException e) { // expected @@ -96,9 +96,9 @@ public void testCompareShardId() { } } for (int i = 1; i < numShardIds - 1; i++) { - assertTrue(DynamodbStreamsShardHandle.compareShardIds(shardIds[i - 1], shardIds[i]) < 0); - assertTrue(DynamodbStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i]) == 0); - assertTrue(DynamodbStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i + 1]) < 0); + assertTrue(DynamoDBStreamsShardHandle.compareShardIds(shardIds[i - 1], shardIds[i]) < 0); + assertTrue(DynamoDBStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i]) == 0); + assertTrue(DynamoDBStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i + 1]) < 0); } } From 45f42e4b9cf9cd493af6e21f7f86aa894ab1227d Mon Sep 17 00:00:00 2001 From: Ying Date: Fri, 21 Dec 2018 14:05:35 -0800 Subject: [PATCH 11/15] FLINK-4582: remove config option; exclude older databind dependency --- flink-connectors/flink-connector-kinesis/pom.xml | 4 ++++ .../kinesis/config/ConsumerConfigConstants.java | 10 ---------- .../kinesis/internals/DynamoDBStreamsDataFetcher.java | 8 +------- 3 files changed, 5 insertions(+), 17 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 63ad2ed45707e..31bc87aab1cdf 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -134,6 +134,10 @@ under the License. com.amazonaws aws-java-sdk-cloudwatch + + com.fasterxml.jackson.core + jackson-databind + diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index a0449e0a57775..00dcd1f7c3040 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -128,13 +128,6 @@ 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"; - /** - * Boolean to indicate whether to compare/enforce shardId format based on the one defined in - * DynamoDBStreamsShardHandle. - */ - public static final String DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK = - "flink.dynamodbstreams.shardid.format.check"; - // ------------------------------------------------------------------------ // Default values for consumer configuration // ------------------------------------------------------------------------ @@ -183,9 +176,6 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { public static final long DEFAULT_SHARD_IDLE_INTERVAL_MILLIS = -1; - // By default disable shardId format check. - public static final String DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK = "false"; - /** * To avoid shard iterator expires in {@link ShardConsumer}s, the value for the configured * getRecords interval can not exceed 5 minutes, which is the expire time for retrieved iterators. diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java index b095fb2a4c37c..c2b7be352b1cd 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/DynamoDBStreamsDataFetcher.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.KinesisShardAssigner; -import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.metrics.ShardMetricsReporter; import org.apache.flink.streaming.connectors.kinesis.model.DynamoDBStreamsShardHandle; import org.apache.flink.streaming.connectors.kinesis.model.SequenceNumber; @@ -70,16 +69,11 @@ public DynamoDBStreamsDataFetcher(List streams, createInitialSubscribedStreamsToLastDiscoveredShardsState(streams), // use DynamoDBStreamsProxy DynamoDBStreamsProxy::create); - - shardIdFormatCheck = Boolean.valueOf(configProps.getProperty( - ConsumerConfigConstants.DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK, - ConsumerConfigConstants.DEFAULT_DYNAMODB_STREAMS_SHARDID_FORMAT_CHECK)); } @Override protected boolean shouldAdvanceLastDiscoveredShardId(String shardId, String lastSeenShardIdOfStream) { - if (shardIdFormatCheck && - DynamoDBStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { + if (DynamoDBStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { // shardID update is valid only if the given shard id is greater // than the previous last seen shard id of the stream. return false; From ab5368ebad81c306b81700912cad37a1cce2429c Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Fri, 21 Dec 2018 18:03:24 -0800 Subject: [PATCH 12/15] Update ConsumerConfigConstants.java --- .../kinesis/config/ConsumerConfigConstants.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java index 00dcd1f7c3040..41ac6b877a954 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java @@ -65,16 +65,13 @@ public SentinelSequenceNumber toSentinelSequenceNumber() { /** The date format of initial timestamp to start reading Kinesis stream from (when AT_TIMESTAMP is set for STREAM_INITIAL_POSITION). */ public static final String STREAM_TIMESTAMP_DATE_FORMAT = "flink.stream.initpos.timestamp.format"; - /** The base backoff time between each describeStream attempt (Used for consuming data from DynamoDB streams). */ + /** The base backoff time between each describeStream attempt (for consuming from DynamoDB streams). */ public static final String STREAM_DESCRIBE_BACKOFF_BASE = "flink.stream.describe.backoff.base"; - /** The maximum backoff time between each describeStream attempt (Used for consuming data from DynamoDB streams). */ + /** The maximum backoff time between each describeStream attempt (for consuming from DynamoDB streams). */ public static final String STREAM_DESCRIBE_BACKOFF_MAX = "flink.stream.describe.backoff.max"; - /** - * The power constant for exponential backoff between each describeStream attempt (Used for consuming data from - * DynamoDB streams). - */ + /** The power constant for exponential backoff between each describeStream attempt (for consuming from DynamoDB streams). */ public static final String STREAM_DESCRIBE_BACKOFF_EXPONENTIAL_CONSTANT = "flink.stream.describe.backoff.expconst"; /** The maximum number of listShards attempts if we get a recoverable exception. */ From 219519925c23c1ec3340b50d984d6f4631de616d Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Fri, 21 Dec 2018 18:05:53 -0800 Subject: [PATCH 13/15] Update FlinkDynamoDBStreamsConsumer.java --- .../connectors/kinesis/FlinkDynamoDBStreamsConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java index c1395bd794565..fd920c4b13a0a 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkDynamoDBStreamsConsumer.java @@ -31,7 +31,7 @@ import java.util.Properties; /** - * Consume events from the dyanmodbdb streams. + * Consume events from DynamoDB streams. * * @param the type of data emitted */ From d43ba9e0883cd8c9db9beb99485090eb839b8b8b Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Fri, 21 Dec 2018 18:10:29 -0800 Subject: [PATCH 14/15] Update KinesisProxy.java --- .../flink/streaming/connectors/kinesis/proxy/KinesisProxy.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index 10a4774345b46..4e44aebd3771e 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -129,7 +129,6 @@ public class KinesisProxy implements KinesisProxyInterface { /** Maximum retry attempts for the get shard iterator operation. */ private final int getShardIteratorMaxRetries; - /* Backoff millis for the describe stream operation. */ private final long describeStreamBaseBackoffMillis; From e89b4a540e9954560d399cfed425f1132b844597 Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Fri, 21 Dec 2018 21:47:56 -0800 Subject: [PATCH 15/15] fixup --- flink-connectors/flink-connector-kinesis/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 31bc87aab1cdf..6c4567751d36a 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -140,6 +140,7 @@ under the License. +