├── .gitignore ├── amazon-kinesis-connector-flink └── src │ ├── main │ ├── resources │ │ ├── software │ │ │ └── amazon │ │ │ │ └── kinesis │ │ │ │ └── shaded │ │ │ │ └── software │ │ │ │ └── amazon │ │ │ │ └── awssdk │ │ │ │ └── global │ │ │ │ └── handlers │ │ │ │ └── execution.interceptors │ │ └── META-INF │ │ │ ├── services │ │ │ ├── software.amazon.kinesis.shaded.software.amazon.awssdk.http.SdkHttpService │ │ │ └── org.apache.flink.table.factories.Factory │ │ │ └── licenses │ │ │ └── LICENSE.protobuf │ └── java │ │ └── software │ │ └── amazon │ │ └── kinesis │ │ └── connectors │ │ └── flink │ │ ├── util │ │ ├── TimeoutLatch.java │ │ ├── KinesisStateUtil.java │ │ ├── BeanDeserializerModifierForIgnorables.java │ │ └── WatermarkTracker.java │ │ ├── RandomKinesisPartitioner.java │ │ ├── FlinkKinesisException.java │ │ ├── serialization │ │ ├── DynamoDBStreamsSchema.java │ │ ├── KinesisSerializationSchema.java │ │ ├── KinesisDeserializationSchemaWrapper.java │ │ └── KinesisDeserializationSchema.java │ │ ├── KinesisPartitioner.java │ │ ├── internals │ │ ├── publisher │ │ │ ├── RecordPublisherFactory.java │ │ │ ├── RecordPublisher.java │ │ │ ├── polling │ │ │ │ ├── PollingRecordPublisherConfiguration.java │ │ │ │ └── PollingRecordPublisherFactory.java │ │ │ ├── RecordBatch.java │ │ │ └── fanout │ │ │ │ └── FanOutRecordPublisherFactory.java │ │ └── DynamoDBStreamsDataFetcher.java │ │ ├── metrics │ │ ├── KinesisConsumerMetricConstants.java │ │ ├── PollingRecordPublisherMetricsReporter.java │ │ └── ShardConsumerMetricsReporter.java │ │ ├── config │ │ └── ProducerConfigConstants.java │ │ ├── proxy │ │ ├── FullJitterBackoff.java │ │ ├── KinesisProxyV2Interface.java │ │ ├── GetShardListResult.java │ │ ├── KinesisProxyV2Factory.java │ │ └── KinesisProxyInterface.java │ │ ├── KinesisShardAssigner.java │ │ ├── FixedKinesisPartitioner.java │ │ ├── model │ │ ├── DynamoDBStreamsShardHandle.java │ │ ├── SentinelSequenceNumber.java │ │ ├── KinesisStreamShardState.java │ │ ├── SequenceNumber.java │ │ └── StreamShardHandle.java │ │ ├── table │ │ └── RowDataKinesisDeserializationSchema.java │ │ └── FlinkDynamoDBStreamsConsumer.java │ └── test │ ├── resources │ ├── profile │ ├── META-INF │ │ └── services │ │ │ └── org.apache.flink.table.factories.Factory │ └── log4j2-test.properties │ └── java │ └── software │ └── amazon │ └── kinesis │ └── connectors │ └── flink │ ├── testutils │ ├── KinesisShardIdGenerator.java │ ├── TestableFlinkKinesisConsumer.java │ ├── AlwaysThrowsDeserializationSchema.java │ ├── TestSourceContext.java │ ├── TableOptionsBuilder.java │ ├── TestRuntimeContext.java │ ├── TestableKinesisDataFetcherForShardConsumerException.java │ └── KinesisEventsGeneratorProducerThread.java │ ├── model │ ├── SentinelSequenceNumberTest.java │ ├── StreamShardHandleTest.java │ ├── StartingPositionTest.java │ └── DynamoDBStreamsShardHandleTest.java │ ├── examples │ ├── ConsumeFromKinesis.java │ ├── ConsumeFromDynamoDBStreams.java │ └── ProduceIntoKinesis.java │ ├── KinesisConsumerTest.java │ ├── metrics │ ├── PollingRecordPublisherMetricsReporterTest.java │ └── ShardConsumerMetricsReporterTest.java │ ├── internals │ ├── DynamoDBStreamsDataFetcherTest.java │ └── publisher │ │ ├── polling │ │ ├── PollingRecordPublisherFactoryTest.java │ │ └── PollingRecordPublisherConfigurationTest.java │ │ └── RecordBatchTest.java │ ├── util │ ├── JobManagerWatermarkTrackerTest.java │ ├── WatermarkTrackerTest.java │ └── StreamConsumerRegistrarUtilTest.java │ ├── manualtests │ └── ManualProducerTest.java │ └── proxy │ └── KinesisProxyV2FactoryTest.java ├── CODE_OF_CONDUCT.md ├── amazon-kinesis-sql-connector-flink ├── src │ └── main │ │ └── resources │ │ └── META-INF │ │ └── NOTICE └── pom.xml ├── tools └── maven │ └── suppressions.xml ├── CONTRIBUTING.md └── pom.xml /.gitignore: -------------------------------------------------------------------------------- 1 | *.iml 2 | .idea 3 | target 4 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/resources/software/amazon/kinesis/shaded/software/amazon/awssdk/global/handlers/execution.interceptors: -------------------------------------------------------------------------------- 1 | software.amazon.kinesis.shaded.software.amazon.awssdk.core.internal.interceptor.HttpChecksumRequiredInterceptor -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/resources/profile: -------------------------------------------------------------------------------- 1 | [default] 2 | aws_access_key_id=11111111111111111111 3 | aws_secret_access_key=wJalrXUtnFEMI/K7MDENG/bPxRfiCY1111111111 4 | 5 | [foo] 6 | aws_access_key_id=22222222222222222222 7 | aws_secret_access_key=wJalrXUtnFEMI/K7MDENG/bPxRfiCY2222222222 8 | -------------------------------------------------------------------------------- /CODE_OF_CONDUCT.md: -------------------------------------------------------------------------------- 1 | ## Code of Conduct 2 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct). 3 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact 4 | opensource-codeofconduct@amazon.com with any additional questions or comments. 5 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/resources/META-INF/services/software.amazon.kinesis.shaded.software.amazon.awssdk.http.SdkHttpService: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"). 5 | # You may not use this file except in compliance with the License. 6 | # You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | # 16 | 17 | software.amazon.kinesis.shaded.software.amazon.awssdk.http.apache.ApacheSdkHttpService -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | software.amazon.kinesis.connectors.flink.testutils.TestFormatFactory -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | software.amazon.kinesis.connectors.flink.table.KinesisDynamicTableFactory 17 | -------------------------------------------------------------------------------- /amazon-kinesis-sql-connector-flink/src/main/resources/META-INF/NOTICE: -------------------------------------------------------------------------------- 1 | // ------------------------------------------------------------------ 2 | // NOTICE file corresponding to the section 4d of The Apache License, 3 | // ------------------------------------------------------------------ 4 | 5 | Modifications copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. 6 | 7 | flink-sql-connector-kinesis 8 | Copyright 2014-2021 The Apache Software Foundation 9 | 10 | This product includes software developed at 11 | The Apache Software Foundation (http://www.apache.org/). 12 | 13 | This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) 14 | 15 | - joda-time:joda-time:2.5 16 | - commons-io:commons-io:2.4 17 | - commons-lang:commons-lang:2.6 18 | - commons-logging:commons-logging:1.1.3 19 | - commons-codec:commons-codec:1.10 20 | - org.apache.commons:commons-lang3:3.3.2 21 | - com.google.guava:guava:30.0-jre 22 | - com.google.guava:failureaccess:1.0.1 23 | - com.fasterxml.jackson.core:jackson-annotations:2.12.7 24 | - com.fasterxml.jackson.core:jackson-databind:2.12.7 25 | - com.fasterxml.jackson.core:jackson-core:2.12.7 26 | - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.7 -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/KinesisShardIdGenerator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | /** 23 | * A generator for Kinesis shard IDs. 24 | * 25 | *

Kinesis shard ids are in the form of: shardId-\d{12} 26 | */ 27 | public class KinesisShardIdGenerator { 28 | public static String generateFromShardOrder(int order) { 29 | return String.format("shardId-%012d", order); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 22 | 23 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | 30 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/model/SentinelSequenceNumberTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.junit.Test; 23 | 24 | import static org.junit.Assert.assertTrue; 25 | 26 | /** 27 | * Tests for {@link SentinelSequenceNumber}. 28 | */ 29 | public class SentinelSequenceNumberTest { 30 | 31 | @Test 32 | public void allSentinelNumbersAreRecognized() { 33 | for (SentinelSequenceNumber sentinel : SentinelSequenceNumber.values()) { 34 | assertTrue(SentinelSequenceNumber.isSentinelSequenceNumber(sentinel.get())); 35 | } 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/util/TimeoutLatch.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink.util; 22 | 23 | import org.apache.flink.annotation.Internal; 24 | 25 | /** 26 | * Internal use. 27 | */ 28 | @Internal 29 | public class TimeoutLatch { 30 | 31 | private final Object lock = new Object(); 32 | private volatile boolean waiting; 33 | 34 | public void await(long timeout) throws InterruptedException { 35 | synchronized (lock) { 36 | waiting = true; 37 | lock.wait(timeout); 38 | } 39 | } 40 | 41 | public void trigger() { 42 | if (waiting) { 43 | synchronized (lock) { 44 | waiting = false; 45 | lock.notifyAll(); 46 | } 47 | } 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/RandomKinesisPartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink; 22 | 23 | import org.apache.flink.annotation.PublicEvolving; 24 | 25 | import java.util.UUID; 26 | 27 | /** 28 | * A {@link KinesisPartitioner} that maps an arbitrary input {@code element} to a random partition 29 | * ID. 30 | * 31 | * @param The input element type. 32 | */ 33 | @PublicEvolving 34 | public final class RandomKinesisPartitioner extends KinesisPartitioner { 35 | @Override 36 | public String getPartitionId(T element) { 37 | return UUID.randomUUID().toString(); 38 | } 39 | 40 | @Override 41 | public boolean equals(Object o) { 42 | return o instanceof RandomKinesisPartitioner; 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return RandomKinesisPartitioner.class.hashCode(); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/model/StreamShardHandleTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.junit.Test; 23 | 24 | import static org.hamcrest.Matchers.equalTo; 25 | import static org.hamcrest.Matchers.greaterThan; 26 | import static org.hamcrest.Matchers.lessThan; 27 | import static org.junit.Assert.assertThat; 28 | 29 | /** 30 | * Test for methods in the {@link StreamShardHandle} class. 31 | */ 32 | public class StreamShardHandleTest { 33 | @Test 34 | public void testCompareShardIds() { 35 | assertThat(StreamShardHandle.compareShardIds("shardId-000000000001", "shardId-000000000010"), lessThan(0)); 36 | assertThat(StreamShardHandle.compareShardIds("shardId-000000000010", "shardId-000000000010"), equalTo(0)); 37 | assertThat(StreamShardHandle.compareShardIds("shardId-000000000015", "shardId-000000000010"), greaterThan(0)); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/resources/META-INF/licenses/LICENSE.protobuf: -------------------------------------------------------------------------------- 1 | Copyright 2008 Google Inc. All rights reserved. 2 | 3 | Redistribution and use in source and binary forms, with or without 4 | modification, are permitted provided that the following conditions are 5 | met: 6 | 7 | * Redistributions of source code must retain the above copyright 8 | notice, this list of conditions and the following disclaimer. 9 | * Redistributions in binary form must reproduce the above 10 | copyright notice, this list of conditions and the following disclaimer 11 | in the documentation and/or other materials provided with the 12 | distribution. 13 | * Neither the name of Google Inc. nor the names of its 14 | contributors may be used to endorse or promote products derived from 15 | this software without specific prior written permission. 16 | 17 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS 18 | "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT 19 | LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR 20 | A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 21 | OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 22 | SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT 23 | LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, 24 | DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY 25 | THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT 26 | (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE 27 | OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 28 | 29 | Code generated by the Protocol Buffer compiler is owned by the owner 30 | of the input file used when generating it. This code is not 31 | standalone and requires a support library to be linked with it. This 32 | support library is itself covered by the above license. 33 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/FlinkKinesisException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | /** 25 | * A {@link RuntimeException} wrapper indicating the exception was thrown from this connector. 26 | * This class is abstract, semantic subclasses should be created to indicate the type of exception. 27 | */ 28 | @Internal 29 | public abstract class FlinkKinesisException extends RuntimeException { 30 | 31 | public FlinkKinesisException(final String message) { 32 | super(message); 33 | } 34 | 35 | public FlinkKinesisException(final String message, final Throwable cause) { 36 | super(message, cause); 37 | } 38 | 39 | /** 40 | * A semantic {@link RuntimeException} thrown to indicate timeout errors in the Kinesis connector. 41 | */ 42 | @Internal 43 | public static class FlinkKinesisTimeoutException extends FlinkKinesisException { 44 | 45 | public FlinkKinesisTimeoutException(String message) { 46 | super(message); 47 | } 48 | } 49 | 50 | } 51 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/serialization/DynamoDBStreamsSchema.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.serialization; 21 | 22 | import org.apache.flink.api.common.typeinfo.TypeInformation; 23 | 24 | import com.amazonaws.services.dynamodbv2.model.Record; 25 | import com.amazonaws.services.dynamodbv2.streamsadapter.model.RecordObjectMapper; 26 | import com.fasterxml.jackson.databind.ObjectMapper; 27 | 28 | import java.io.IOException; 29 | 30 | /** 31 | * Schema used for deserializing DynamoDB streams records. 32 | */ 33 | public class DynamoDBStreamsSchema implements KinesisDeserializationSchema { 34 | private static final ObjectMapper MAPPER = new RecordObjectMapper(); 35 | 36 | @Override 37 | public Record deserialize(byte[] message, String partitionKey, String seqNum, 38 | long approxArrivalTimestamp, String stream, String shardId) throws IOException { 39 | return MAPPER.readValue(message, Record.class); 40 | } 41 | 42 | @Override 43 | public TypeInformation getProducedType() { 44 | return TypeInformation.of(Record.class); 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/TestableFlinkKinesisConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | import org.apache.flink.api.common.functions.RuntimeContext; 23 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 24 | import org.apache.flink.streaming.util.MockStreamingRuntimeContext; 25 | 26 | import software.amazon.kinesis.connectors.flink.FlinkKinesisConsumer; 27 | 28 | import java.util.Properties; 29 | 30 | /** 31 | * Extension of the {@link FlinkKinesisConsumer} for testing. 32 | */ 33 | public class TestableFlinkKinesisConsumer extends FlinkKinesisConsumer { 34 | 35 | private final RuntimeContext mockedRuntimeCtx; 36 | 37 | public TestableFlinkKinesisConsumer(String fakeStream, 38 | Properties fakeConfiguration, 39 | final int totalNumOfConsumerSubtasks, 40 | final int indexOfThisConsumerSubtask) { 41 | super(fakeStream, new SimpleStringSchema(), fakeConfiguration); 42 | 43 | this.mockedRuntimeCtx = new MockStreamingRuntimeContext(true, totalNumOfConsumerSubtasks, indexOfThisConsumerSubtask); 44 | } 45 | 46 | @Override 47 | public RuntimeContext getRuntimeContext() { 48 | return this.mockedRuntimeCtx; 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/KinesisPartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink; 21 | 22 | import org.apache.flink.annotation.PublicEvolving; 23 | 24 | import java.io.Serializable; 25 | 26 | /** 27 | * An interface for partitioning records. 28 | * 29 | * @param record type 30 | */ 31 | @PublicEvolving 32 | public abstract class KinesisPartitioner implements Serializable { 33 | 34 | private static final long serialVersionUID = -7467294664702189780L; 35 | 36 | /** 37 | * Return a partition id based on the input. 38 | * 39 | * @param element Element to partition 40 | * 41 | * @return A string representing the partition id 42 | */ 43 | public abstract String getPartitionId(T element); 44 | 45 | /** 46 | * Optional method for setting an explicit hash key. 47 | * 48 | * @param element Element to get the hash key for 49 | * 50 | * @return the hash key for the element 51 | */ 52 | public String getExplicitHashKey(T element) { 53 | return null; 54 | } 55 | 56 | /** 57 | * Optional initializer. 58 | * 59 | * @param indexOfThisSubtask Index of this partitioner instance 60 | * @param numberOfParallelSubtasks Total number of parallel instances 61 | */ 62 | public void initialize(int indexOfThisSubtask, int numberOfParallelSubtasks) { 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/publisher/RecordPublisherFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.metrics.MetricGroup; 24 | 25 | import software.amazon.kinesis.connectors.flink.model.StartingPosition; 26 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 27 | 28 | import java.util.Properties; 29 | 30 | /** 31 | * A factory interface used to create instances of {@link RecordPublisher}. 32 | */ 33 | @Internal 34 | public interface RecordPublisherFactory { 35 | 36 | /** 37 | * Create a {@link RecordPublisher}. 38 | * 39 | * @param startingPosition the position in the shard to start consuming records from 40 | * @param consumerConfig the properties used to configure the {@link RecordPublisher}. 41 | * @param metricGroup the {@link MetricGroup} used to report metrics to 42 | * @param streamShardHandle the stream shard in which to consume from 43 | * @return the constructed {@link RecordPublisher} 44 | */ 45 | RecordPublisher create( 46 | StartingPosition startingPosition, 47 | Properties consumerConfig, 48 | MetricGroup metricGroup, 49 | StreamShardHandle streamShardHandle) throws InterruptedException; 50 | 51 | /** 52 | * Destroy any open resources used by the factory. 53 | */ 54 | default void close() { 55 | // Do nothing by default 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/metrics/KinesisConsumerMetricConstants.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink.metrics; 22 | 23 | import org.apache.flink.annotation.Internal; 24 | 25 | /** 26 | * A collection of consumer metric related constant names. 27 | * 28 | *

The names must not be changed, as that would break backwards compatibility for the consumer metrics. 29 | */ 30 | @Internal 31 | public class KinesisConsumerMetricConstants { 32 | 33 | public static final String KINESIS_CONSUMER_METRICS_GROUP = "KinesisConsumer"; 34 | 35 | public static final String STREAM_METRICS_GROUP = "stream"; 36 | public static final String SHARD_METRICS_GROUP = "shardId"; 37 | 38 | public static final String MILLIS_BEHIND_LATEST_GAUGE = "millisBehindLatest"; 39 | public static final String SLEEP_TIME_MILLIS = "sleepTimeMillis"; 40 | public static final String MAX_RECORDS_PER_FETCH = "maxNumberOfRecordsPerFetch"; 41 | public static final String NUM_AGGREGATED_RECORDS_PER_FETCH = "numberOfAggregatedRecordsPerFetch"; 42 | public static final String NUM_DEAGGREGATED_RECORDS_PER_FETCH = "numberOfDeaggregatedRecordsPerFetch"; 43 | public static final String AVG_RECORD_SIZE_BYTES = "averageRecordSizeBytes"; 44 | public static final String RUNTIME_LOOP_NANOS = "runLoopTimeNanos"; 45 | public static final String LOOP_FREQUENCY_HZ = "loopFrequencyHz"; 46 | public static final String BYTES_PER_READ = "bytesRequestedPerFetch"; 47 | 48 | } 49 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/config/ProducerConfigConstants.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.config; 21 | 22 | import software.amazon.kinesis.connectors.flink.FlinkKinesisProducer; 23 | 24 | /** 25 | * Optional producer specific configuration keys for {@link FlinkKinesisProducer}. 26 | * 27 | * @deprecated This class is deprecated in favor of the official AWS Kinesis producer configuration keys. 28 | * See 29 | * here for the full list of available configs. 30 | * For configuring the region and credentials, please use the keys in {@link AWSConfigConstants}. 31 | */ 32 | @Deprecated 33 | public class ProducerConfigConstants extends AWSConfigConstants { 34 | 35 | /** 36 | * Deprecated key. 37 | * 38 | * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration keys. 39 | * Please use {@code CollectionMaxCount} instead. 40 | **/ 41 | @Deprecated 42 | public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount"; 43 | 44 | /** 45 | * Deprecated key. 46 | * 47 | * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration keys. 48 | * Please use {@code AggregationMaxCount} instead. 49 | **/ 50 | @Deprecated 51 | public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount"; 52 | } 53 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/AlwaysThrowsDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | import org.apache.flink.api.common.serialization.DeserializationSchema; 23 | import org.apache.flink.api.common.typeinfo.BasicTypeInfo; 24 | import org.apache.flink.api.common.typeinfo.TypeInformation; 25 | import org.apache.flink.core.testutils.OneShotLatch; 26 | 27 | import java.io.IOException; 28 | 29 | /** 30 | * A DeserializationSchema which always throws an exception when the deserialize method is called. Also supports 31 | * waiting on a latch until at least one exception has been thrown. 32 | */ 33 | public class AlwaysThrowsDeserializationSchema implements DeserializationSchema { 34 | public static final String EXCEPTION_MESSAGE = "This method always throws an exception."; 35 | 36 | public transient OneShotLatch isExceptionThrown = new OneShotLatch(); 37 | 38 | @Override 39 | public String deserialize(final byte[] bytes) throws IOException { 40 | isExceptionThrown.trigger(); 41 | throw new RuntimeException(EXCEPTION_MESSAGE); 42 | } 43 | 44 | @Override 45 | public boolean isEndOfStream(final String s) { 46 | return false; 47 | } 48 | 49 | @Override 50 | public TypeInformation getProducedType() { 51 | return BasicTypeInfo.STRING_TYPE_INFO; 52 | } 53 | 54 | private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { 55 | in.defaultReadObject(); 56 | this.isExceptionThrown = new OneShotLatch(); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/TestSourceContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 23 | import org.apache.flink.streaming.api.watermark.Watermark; 24 | import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; 25 | 26 | import java.util.concurrent.ConcurrentLinkedQueue; 27 | 28 | /** 29 | * A testable {@link SourceFunction.SourceContext}. 30 | */ 31 | public class TestSourceContext implements SourceFunction.SourceContext { 32 | 33 | private final Object checkpointLock = new Object(); 34 | 35 | private ConcurrentLinkedQueue> collectedOutputs = new ConcurrentLinkedQueue<>(); 36 | 37 | @Override 38 | public void collect(T element) { 39 | this.collectedOutputs.add(new StreamRecord<>(element)); 40 | } 41 | 42 | @Override 43 | public void collectWithTimestamp(T element, long timestamp) { 44 | this.collectedOutputs.add(new StreamRecord<>(element, timestamp)); 45 | } 46 | 47 | @Override 48 | public void emitWatermark(Watermark mark) { 49 | throw new UnsupportedOperationException(); 50 | } 51 | 52 | @Override 53 | public void markAsTemporarilyIdle() {} 54 | 55 | @Override 56 | public Object getCheckpointLock() { 57 | return checkpointLock; 58 | } 59 | 60 | @Override 61 | public void close() {} 62 | 63 | public StreamRecord removeLatestOutput() { 64 | return collectedOutputs.poll(); 65 | } 66 | 67 | public ConcurrentLinkedQueue> getCollectedOutputs() { 68 | return collectedOutputs; 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/examples/ConsumeFromKinesis.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.examples; 21 | 22 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 23 | import org.apache.flink.api.java.utils.ParameterTool; 24 | import org.apache.flink.streaming.api.datastream.DataStream; 25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 26 | 27 | import software.amazon.kinesis.connectors.flink.FlinkKinesisConsumer; 28 | import software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants; 29 | 30 | import java.util.Properties; 31 | 32 | /** 33 | * This is an example on how to consume data from Kinesis. 34 | */ 35 | public class ConsumeFromKinesis { 36 | 37 | public static void main(String[] args) throws Exception { 38 | ParameterTool pt = ParameterTool.fromArgs(args); 39 | 40 | StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); 41 | see.setParallelism(1); 42 | 43 | Properties kinesisConsumerConfig = new Properties(); 44 | kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_REGION, pt.getRequired("region")); 45 | kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accesskey")); 46 | kinesisConsumerConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey")); 47 | 48 | DataStream kinesis = see.addSource(new FlinkKinesisConsumer<>( 49 | "flink-test", 50 | new SimpleStringSchema(), 51 | kinesisConsumerConfig)); 52 | 53 | kinesis.print(); 54 | 55 | see.execute(); 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/publisher/RecordPublisher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import software.amazon.kinesis.connectors.flink.model.SequenceNumber; 25 | 26 | /** 27 | * A {@code RecordPublisher} will consume records from an external stream and deliver them to the registered subscriber. 28 | */ 29 | @Internal 30 | public interface RecordPublisher { 31 | 32 | /** 33 | * Run the record publisher. Records will be consumed from the stream and published to the consumer. 34 | * The number of batches retrieved by a single invocation will vary based on implementation. 35 | * 36 | * @param recordBatchConsumer the record batch consumer in which to output records 37 | * @return a status enum to represent whether a shard has been fully consumed 38 | * @throws InterruptedException 39 | */ 40 | RecordPublisherRunResult run(RecordBatchConsumer recordBatchConsumer) throws InterruptedException; 41 | 42 | /** 43 | * A status enum to represent whether a shard has been fully consumed. 44 | */ 45 | enum RecordPublisherRunResult { 46 | /** There are no more records to consume from this shard. */ 47 | COMPLETE, 48 | 49 | /** There are more records to consume from this shard. */ 50 | INCOMPLETE, 51 | 52 | /** The record publisher has been cancelled. */ 53 | CANCELLED 54 | } 55 | 56 | /** 57 | * An interface used to collect record batches, and reply with the latest consumed sequence number. 58 | */ 59 | interface RecordBatchConsumer { 60 | 61 | SequenceNumber accept(RecordBatch recordBatch); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/serialization/KinesisSerializationSchema.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.serialization; 21 | 22 | import org.apache.flink.annotation.PublicEvolving; 23 | import org.apache.flink.api.common.serialization.SerializationSchema.InitializationContext; 24 | 25 | import java.io.Serializable; 26 | import java.nio.ByteBuffer; 27 | 28 | /** 29 | * Kinesis-specific serialization schema, allowing users to specify a target stream based 30 | * on a record's contents. 31 | * @param 32 | */ 33 | @PublicEvolving 34 | public interface KinesisSerializationSchema extends Serializable { 35 | /** 36 | * Initialization method for the schema. It is called before the actual working methods 37 | * {@link #serialize(Object)} and thus suitable for one time setup work. 38 | * 39 | *

The provided {@link InitializationContext} can be used to access additional features such 40 | * as e.g. registering user metrics. 41 | * 42 | * @param context Contextual information that can be used during initialization. 43 | */ 44 | default void open(InitializationContext context) throws Exception { 45 | } 46 | 47 | /** 48 | * Serialize the given element into a ByteBuffer. 49 | * 50 | * @param element The element to serialize 51 | * @return Serialized representation of the element 52 | */ 53 | ByteBuffer serialize(T element); 54 | 55 | /** 56 | * Optional method to determine the target stream based on the element. 57 | * Return null to use the default stream 58 | * 59 | * @param element The element to determine the target stream from 60 | * @return target stream name 61 | */ 62 | String getTargetStream(T element); 63 | } 64 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/proxy/FullJitterBackoff.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.proxy; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import java.util.Random; 25 | 26 | /** 27 | * Used to calculate full jitter backoff sleep durations. 28 | * @see 29 | * Exponential Backoff and Jitter 30 | * 31 | */ 32 | @Internal 33 | public class FullJitterBackoff { 34 | 35 | /** Random seed used to calculate backoff jitter for Kinesis operations. */ 36 | private final Random seed = new Random(); 37 | 38 | /** 39 | * Calculates the sleep time for full jitter based on the given parameters. 40 | * 41 | * @param baseMillis the base backoff time in milliseconds 42 | * @param maxMillis the maximum backoff time in milliseconds 43 | * @param power the power constant for exponential backoff 44 | * @param attempt the attempt number 45 | * @return the time to wait before trying again 46 | */ 47 | public long calculateFullJitterBackoff(long baseMillis, long maxMillis, double power, int attempt) { 48 | long exponentialBackoff = (long) Math.min(maxMillis, baseMillis * Math.pow(power, attempt)); 49 | return (long) (seed.nextDouble() * exponentialBackoff); 50 | } 51 | 52 | /** 53 | * Puts the current thread to sleep for the specified number of millis. 54 | * Simply delegates to {@link Thread#sleep}. 55 | * 56 | * @param millisToSleep the number of milliseconds to sleep for 57 | * @throws InterruptedException 58 | */ 59 | public void sleep(long millisToSleep) throws InterruptedException { 60 | Thread.sleep(millisToSleep); 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/TableOptionsBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink.testutils; 22 | 23 | import org.apache.flink.configuration.ConfigOption; 24 | import org.apache.flink.table.factories.FactoryUtil; 25 | 26 | import java.util.HashMap; 27 | import java.util.Map; 28 | 29 | /** 30 | * Convenience syntax for constructing option maps for testing 31 | * {@link org.apache.flink.table.factories.DynamicTableFactory} implementations. 32 | *

Back-ported from Flink upstream, branch 1.12.0-rc3, `org.apache.flink.table.factories.TableOptionsBuilder`.

33 | */ 34 | public class TableOptionsBuilder { 35 | private final Map options; 36 | private final String connector; 37 | private final String format; 38 | 39 | public TableOptionsBuilder(String connector, String format) { 40 | this.options = new HashMap<>(); 41 | this.connector = connector; 42 | this.format = format; 43 | } 44 | 45 | public TableOptionsBuilder withTableOption(ConfigOption option, String value) { 46 | return withTableOption(option.key(), value); 47 | } 48 | 49 | public TableOptionsBuilder withFormatOption(ConfigOption option, String value) { 50 | return withFormatOption(format + "." + option.key(), value); 51 | } 52 | 53 | public TableOptionsBuilder withTableOption(String key, String value) { 54 | options.put(key, value); 55 | return this; 56 | } 57 | 58 | public TableOptionsBuilder withFormatOption(String key, String value) { 59 | options.put(key, value); 60 | return this; 61 | } 62 | 63 | public Map build() { 64 | withTableOption(FactoryUtil.CONNECTOR, connector); 65 | withTableOption(FactoryUtil.FORMAT, format); 66 | return options; 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/KinesisShardAssigner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink; 21 | 22 | import org.apache.flink.annotation.PublicEvolving; 23 | 24 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 25 | 26 | import java.io.Serializable; 27 | 28 | /** 29 | * Utility to map Kinesis shards to Flink subtask indices. Users can implement this interface to optimize 30 | * distribution of shards over subtasks. See {@link #assign(StreamShardHandle, int)} for details. 31 | */ 32 | @PublicEvolving 33 | public interface KinesisShardAssigner extends Serializable { 34 | 35 | /** 36 | * Returns the index of the target subtask that a specific shard should be 37 | * assigned to. For return values outside the subtask range, modulus operation will 38 | * be applied automatically, hence it is also valid to just return a hash code. 39 | * 40 | *

The resulting distribution of shards should have the following contract: 41 | *

45 | * 46 | *

The above contract is crucial and cannot be broken. Consumer subtasks rely on this 47 | * contract to filter out shards that they should not subscribe to, guaranteeing 48 | * that each shard of a stream will always be assigned to one subtask in a 49 | * uniformly distributed manner. 50 | * 51 | * @param shard the shard to determine 52 | * @param numParallelSubtasks total number of subtasks 53 | * @return target index, if index falls outside of the range, modulus operation will be applied 54 | */ 55 | int assign(StreamShardHandle shard, int numParallelSubtasks); 56 | } 57 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/publisher/polling/PollingRecordPublisherConfiguration.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher.polling; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants; 25 | 26 | import java.util.Properties; 27 | 28 | /** 29 | * A configuration class for {@link PollingRecordPublisher} instantiated from a properties map. 30 | */ 31 | @Internal 32 | public class PollingRecordPublisherConfiguration { 33 | 34 | private final boolean adaptiveReads; 35 | 36 | private final int maxNumberOfRecordsPerFetch; 37 | 38 | private final long fetchIntervalMillis; 39 | 40 | public PollingRecordPublisherConfiguration(final Properties consumerConfig) { 41 | this.maxNumberOfRecordsPerFetch = Integer.parseInt(consumerConfig.getProperty( 42 | ConsumerConfigConstants.SHARD_GETRECORDS_MAX, 43 | Integer.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_MAX))); 44 | 45 | this.fetchIntervalMillis = Long.parseLong(consumerConfig.getProperty( 46 | ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS, 47 | Long.toString(ConsumerConfigConstants.DEFAULT_SHARD_GETRECORDS_INTERVAL_MILLIS))); 48 | 49 | this.adaptiveReads = Boolean.parseBoolean(consumerConfig.getProperty( 50 | ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS, 51 | Boolean.toString(ConsumerConfigConstants.DEFAULT_SHARD_USE_ADAPTIVE_READS))); 52 | } 53 | 54 | public boolean isAdaptiveReads() { 55 | return adaptiveReads; 56 | } 57 | 58 | public int getMaxNumberOfRecordsPerFetch() { 59 | return maxNumberOfRecordsPerFetch; 60 | } 61 | 62 | public long getFetchIntervalMillis() { 63 | return fetchIntervalMillis; 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/util/KinesisStateUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"). 5 | * You may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package software.amazon.kinesis.connectors.flink.util; 18 | 19 | import org.apache.flink.api.common.ExecutionConfig; 20 | import org.apache.flink.api.common.typeinfo.TypeInformation; 21 | import org.apache.flink.api.common.typeutils.TypeSerializer; 22 | import org.apache.flink.api.java.tuple.Tuple2; 23 | import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; 24 | import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; 25 | 26 | import software.amazon.kinesis.connectors.flink.model.SequenceNumber; 27 | import software.amazon.kinesis.connectors.flink.model.StreamShardMetadata; 28 | 29 | /** 30 | * Utilities for Flink Kinesis connector state management. 31 | */ 32 | public class KinesisStateUtil { 33 | 34 | /** 35 | * To prevent instantiation of class. 36 | */ 37 | private KinesisStateUtil() { 38 | } 39 | 40 | /** 41 | * Creates state serializer for kinesis shard sequence number. 42 | * Using of the explicit state serializer with KryoSerializer is needed because otherwise 43 | * users cannot use 'disableGenericTypes' properties with KinesisConsumer, see FLINK-24943 for details 44 | * 45 | * @return state serializer 46 | */ 47 | public static TupleSerializer> createShardsStateSerializer(ExecutionConfig executionConfig) { 48 | // explicit serializer will keep the compatibility with GenericTypeInformation and allow to disableGenericTypes for users 49 | TypeSerializer[] fieldSerializers = new TypeSerializer[]{ 50 | TypeInformation.of(StreamShardMetadata.class).createSerializer(executionConfig), 51 | new KryoSerializer<>(SequenceNumber.class, executionConfig) 52 | }; 53 | @SuppressWarnings("unchecked") 54 | Class> tupleClass = (Class>) (Class) Tuple2.class; 55 | return new TupleSerializer<>(tupleClass, fieldSerializers); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/examples/ConsumeFromDynamoDBStreams.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.examples; 21 | 22 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 23 | import org.apache.flink.api.java.utils.ParameterTool; 24 | import org.apache.flink.streaming.api.datastream.DataStream; 25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 26 | 27 | import software.amazon.kinesis.connectors.flink.FlinkDynamoDBStreamsConsumer; 28 | import software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants; 29 | 30 | import java.util.Properties; 31 | 32 | /** 33 | * Sample command-line program of consuming data from a single DynamoDB stream. 34 | */ 35 | public class ConsumeFromDynamoDBStreams { 36 | private static final String DYNAMODB_STREAM_NAME = "stream"; 37 | 38 | public static void main(String[] args) throws Exception { 39 | ParameterTool pt = ParameterTool.fromArgs(args); 40 | 41 | StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); 42 | see.setParallelism(1); 43 | 44 | Properties dynamodbStreamsConsumerConfig = new Properties(); 45 | final String streamName = pt.getRequired(DYNAMODB_STREAM_NAME); 46 | dynamodbStreamsConsumerConfig.setProperty( 47 | ConsumerConfigConstants.AWS_REGION, pt.getRequired("region")); 48 | dynamodbStreamsConsumerConfig.setProperty( 49 | ConsumerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accesskey")); 50 | dynamodbStreamsConsumerConfig.setProperty( 51 | ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretkey")); 52 | 53 | DataStream dynamodbStreams = see.addSource(new FlinkDynamoDBStreamsConsumer<>( 54 | streamName, 55 | new SimpleStringSchema(), 56 | dynamodbStreamsConsumerConfig)); 57 | 58 | dynamodbStreams.print(); 59 | 60 | see.execute(); 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/FixedKinesisPartitioner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink; 22 | 23 | import org.apache.flink.annotation.PublicEvolving; 24 | import org.apache.flink.util.Preconditions; 25 | 26 | import java.util.Objects; 27 | 28 | /** 29 | * A partitioner ensuring that each internal Flink partition ends up in the same Kinesis partition. 30 | * 31 | *

This is achieved by using the index of the producer task as a {@code PartitionKey}.

32 | */ 33 | @PublicEvolving 34 | public final class FixedKinesisPartitioner extends KinesisPartitioner { 35 | 36 | private static final long serialVersionUID = 1L; 37 | 38 | private int indexOfThisSubtask = 0; 39 | 40 | @Override 41 | public void initialize(int indexOfThisSubtask, int numberOfParallelSubtasks) { 42 | Preconditions.checkArgument( 43 | indexOfThisSubtask >= 0, 44 | "Id of this subtask cannot be negative."); 45 | Preconditions.checkArgument( 46 | numberOfParallelSubtasks > 0, 47 | "Number of subtasks must be larger than 0."); 48 | 49 | this.indexOfThisSubtask = indexOfThisSubtask; 50 | } 51 | 52 | @Override 53 | public String getPartitionId(T record) { 54 | return String.valueOf(indexOfThisSubtask); 55 | } 56 | 57 | // -------------------------------------------------------------------------------------------- 58 | // Value semantics for equals and hashCode 59 | // -------------------------------------------------------------------------------------------- 60 | 61 | @Override 62 | public boolean equals(Object o) { 63 | if (this == o) { 64 | return true; 65 | } 66 | if (o == null || getClass() != o.getClass()) { 67 | return false; 68 | } 69 | final FixedKinesisPartitioner that = (FixedKinesisPartitioner) o; 70 | return Objects.equals(this.indexOfThisSubtask, that.indexOfThisSubtask); 71 | } 72 | 73 | @Override 74 | public int hashCode() { 75 | return Objects.hash( 76 | FixedKinesisPartitioner.class.hashCode(), 77 | indexOfThisSubtask); 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/KinesisConsumerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink; 22 | 23 | import org.apache.flink.api.common.serialization.DeserializationSchema; 24 | import org.apache.flink.api.common.typeinfo.TypeInformation; 25 | import org.apache.flink.util.Collector; 26 | 27 | import org.junit.Rule; 28 | import org.junit.Test; 29 | import org.junit.rules.ExpectedException; 30 | 31 | import java.io.IOException; 32 | import java.util.Properties; 33 | 34 | /** 35 | * Tests for {@link FlinkKinesisConsumer}. In contrast to tests in {@link FlinkKinesisConsumerTest} it does not 36 | * use power mock, which makes it possible to use e.g. the {@link ExpectedException}. 37 | */ 38 | public class KinesisConsumerTest { 39 | 40 | @Rule 41 | public ExpectedException thrown = ExpectedException.none(); 42 | 43 | @Test 44 | public void testKinesisConsumerThrowsExceptionIfSchemaImplementsCollector() { 45 | DeserializationSchema schemaWithCollector = new DeserializationSchema() { 46 | @Override 47 | public Object deserialize(byte[] message) throws IOException { 48 | return null; 49 | } 50 | 51 | @Override 52 | public void deserialize(byte[] message, Collector out) throws IOException { 53 | // we do not care about the implementation. we should just check if this method is declared 54 | } 55 | 56 | @Override 57 | public boolean isEndOfStream(Object nextElement) { 58 | return false; 59 | } 60 | 61 | @Override 62 | public TypeInformation getProducedType() { 63 | return null; 64 | } 65 | }; 66 | 67 | thrown.expect(IllegalArgumentException.class); 68 | thrown.expectMessage( 69 | "Kinesis consumer does not support DeserializationSchema that implements deserialization with a" + 70 | " Collector. Unsupported DeserializationSchema: " + 71 | "software.amazon.kinesis.connectors.flink.KinesisConsumerTest"); 72 | new FlinkKinesisConsumer<>("fakeStream", schemaWithCollector, new Properties()); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/model/DynamoDBStreamsShardHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import com.amazonaws.services.kinesis.model.Shard; 23 | 24 | /** 25 | * DynamoDB streams shard handle format and utilities. 26 | */ 27 | public class DynamoDBStreamsShardHandle extends StreamShardHandle{ 28 | public static final String SHARDID_PREFIX = "shardId-"; 29 | public static final int SHARDID_PREFIX_LEN = SHARDID_PREFIX.length(); 30 | 31 | public DynamoDBStreamsShardHandle(String streamName, Shard shard) { 32 | super(streamName, shard); 33 | } 34 | 35 | public static int compareShardIds(String firstShardId, String secondShardId) { 36 | if (!isValidShardId(firstShardId)) { 37 | throw new IllegalArgumentException( 38 | String.format("The first shard id %s has invalid format.", firstShardId)); 39 | } else if (!isValidShardId(secondShardId)) { 40 | throw new IllegalArgumentException( 41 | String.format("The second shard id %s has invalid format.", secondShardId)); 42 | } 43 | 44 | return firstShardId.substring(SHARDID_PREFIX_LEN).compareTo( 45 | secondShardId.substring(SHARDID_PREFIX_LEN)); 46 | } 47 | 48 | /** 49 | *

50 | * Dynamodb streams shard ID is a char string ranging from 28 characters to 65 characters. 51 | * (See https://docs.aws.amazon.com/amazondynamodb/latest/APIReference/API_streams_Shard.html) 52 | * 53 | * The shardId observed usually takes the format of: "shardId-00000001536805703746-69688cb1", 54 | * where "shardId-" is a prefix, followed by a 20-digit timestamp string and 0-36 or more 55 | * characters, separated by '-'. Following this format, it is expected the child shards created 56 | * during a re-sharding event have shardIds bigger than their parents. 57 | *

58 | * @param shardId shard Id 59 | * @return boolean indicate if the given shard Id is valid 60 | */ 61 | public static boolean isValidShardId(String shardId) { 62 | return shardId == null ? false : shardId.matches("^shardId-\\d{20}-{0,1}\\w{0,36}"); 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/proxy/KinesisProxyV2Interface.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.proxy; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import software.amazon.awssdk.services.kinesis.model.DeregisterStreamConsumerResponse; 25 | import software.amazon.awssdk.services.kinesis.model.DescribeStreamConsumerResponse; 26 | import software.amazon.awssdk.services.kinesis.model.DescribeStreamSummaryResponse; 27 | import software.amazon.awssdk.services.kinesis.model.RegisterStreamConsumerResponse; 28 | import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; 29 | import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; 30 | 31 | import java.util.concurrent.CompletableFuture; 32 | import java.util.concurrent.ExecutionException; 33 | 34 | /** 35 | * Interface for a Kinesis proxy using AWS SDK v2.x operating on multiple Kinesis streams within the same AWS service region. 36 | */ 37 | @Internal 38 | public interface KinesisProxyV2Interface { 39 | 40 | DescribeStreamSummaryResponse describeStreamSummary(String stream) throws InterruptedException, ExecutionException; 41 | 42 | DescribeStreamConsumerResponse describeStreamConsumer(final String streamConsumerArn) throws InterruptedException, ExecutionException; 43 | 44 | DescribeStreamConsumerResponse describeStreamConsumer(final String streamArn, final String consumerName) throws InterruptedException, ExecutionException; 45 | 46 | RegisterStreamConsumerResponse registerStreamConsumer(final String streamArn, final String consumerName) throws InterruptedException, ExecutionException; 47 | 48 | DeregisterStreamConsumerResponse deregisterStreamConsumer(final String consumerArn) throws InterruptedException, ExecutionException; 49 | 50 | CompletableFuture subscribeToShard(SubscribeToShardRequest request, SubscribeToShardResponseHandler responseHandler); 51 | 52 | /** 53 | * Destroy any open resources used by the factory. 54 | */ 55 | default void close() { 56 | // Do nothing by default 57 | } 58 | 59 | } 60 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/metrics/PollingRecordPublisherMetricsReporterTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.metrics; 21 | 22 | import org.apache.flink.metrics.MetricGroup; 23 | 24 | import org.junit.Before; 25 | import org.junit.Test; 26 | import org.mockito.InjectMocks; 27 | import org.mockito.Mock; 28 | import org.mockito.MockitoAnnotations; 29 | 30 | import static org.junit.Assert.assertEquals; 31 | import static org.mockito.ArgumentMatchers.any; 32 | import static org.mockito.ArgumentMatchers.eq; 33 | import static org.mockito.Mockito.verify; 34 | 35 | /** 36 | * Tests for {@link PollingRecordPublisherMetricsReporter}. 37 | */ 38 | public class PollingRecordPublisherMetricsReporterTest { 39 | 40 | @InjectMocks 41 | private PollingRecordPublisherMetricsReporter metricsReporter; 42 | 43 | @Mock 44 | private MetricGroup metricGroup; 45 | 46 | @Before 47 | public void setUp() { 48 | MockitoAnnotations.initMocks(this); 49 | } 50 | 51 | @Test 52 | public void testMetricIdentifiers() { 53 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.BYTES_PER_READ), any()); 54 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.LOOP_FREQUENCY_HZ), any()); 55 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.MAX_RECORDS_PER_FETCH), any()); 56 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.RUNTIME_LOOP_NANOS), any()); 57 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.SLEEP_TIME_MILLIS), any()); 58 | } 59 | 60 | @Test 61 | public void testGettersAndSetters() { 62 | metricsReporter.setBytesPerRead(1); 63 | metricsReporter.setLoopFrequencyHz(2); 64 | metricsReporter.setMaxNumberOfRecordsPerFetch(3); 65 | metricsReporter.setRunLoopTimeNanos(4); 66 | metricsReporter.setSleepTimeMillis(5); 67 | 68 | assertEquals(1, metricsReporter.getBytesPerRead(), 0); 69 | assertEquals(2, metricsReporter.getLoopFrequencyHz(), 0); 70 | assertEquals(3, metricsReporter.getMaxNumberOfRecordsPerFetch()); 71 | assertEquals(4, metricsReporter.getRunLoopTimeNanos()); 72 | assertEquals(5, metricsReporter.getSleepTimeMillis()); 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/proxy/GetShardListResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.proxy; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 25 | 26 | import java.util.HashMap; 27 | import java.util.LinkedList; 28 | import java.util.List; 29 | import java.util.Map; 30 | import java.util.Set; 31 | 32 | /** 33 | * Basic model class to bundle the shards retrieved from Kinesis on a {@link KinesisProxyInterface#getShardList(Map)} call. 34 | */ 35 | @Internal 36 | public class GetShardListResult { 37 | 38 | private final Map> streamsToRetrievedShardList = new HashMap<>(); 39 | 40 | public void addRetrievedShardToStream(String stream, StreamShardHandle retrievedShard) { 41 | if (!streamsToRetrievedShardList.containsKey(stream)) { 42 | streamsToRetrievedShardList.put(stream, new LinkedList()); 43 | } 44 | streamsToRetrievedShardList.get(stream).add(retrievedShard); 45 | } 46 | 47 | public void addRetrievedShardsToStream(String stream, List retrievedShards) { 48 | if (retrievedShards.size() != 0) { 49 | if (!streamsToRetrievedShardList.containsKey(stream)) { 50 | streamsToRetrievedShardList.put(stream, new LinkedList()); 51 | } 52 | streamsToRetrievedShardList.get(stream).addAll(retrievedShards); 53 | } 54 | } 55 | 56 | public List getRetrievedShardListOfStream(String stream) { 57 | if (!streamsToRetrievedShardList.containsKey(stream)) { 58 | return null; 59 | } else { 60 | return streamsToRetrievedShardList.get(stream); 61 | } 62 | } 63 | 64 | public StreamShardHandle getLastSeenShardOfStream(String stream) { 65 | if (!streamsToRetrievedShardList.containsKey(stream)) { 66 | return null; 67 | } else { 68 | return streamsToRetrievedShardList.get(stream).getLast(); 69 | } 70 | } 71 | 72 | public boolean hasRetrievedShards() { 73 | return !streamsToRetrievedShardList.isEmpty(); 74 | } 75 | 76 | public Set getStreamsWithRetrievedShards() { 77 | return streamsToRetrievedShardList.keySet(); 78 | } 79 | 80 | } 81 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/util/BeanDeserializerModifierForIgnorables.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.util; 21 | 22 | import com.fasterxml.jackson.databind.BeanDescription; 23 | import com.fasterxml.jackson.databind.DeserializationConfig; 24 | import com.fasterxml.jackson.databind.deser.BeanDeserializerBuilder; 25 | import com.fasterxml.jackson.databind.deser.BeanDeserializerModifier; 26 | import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition; 27 | 28 | import java.util.ArrayList; 29 | import java.util.List; 30 | 31 | /** 32 | * Jackson bean deserializer utility that allows skipping of properties, for example because they 33 | * cannot be handled by the default serializer or should be ignored for other reason. 34 | * 35 | *

Original source: 36 | * https://stackoverflow.com/questions/12305438/jackson-dynamic-filtering-of-properties-during-deserialization 37 | */ 38 | public class BeanDeserializerModifierForIgnorables extends BeanDeserializerModifier { 39 | 40 | private Class type; 41 | private List ignorables; 42 | 43 | public BeanDeserializerModifierForIgnorables(Class clazz, String... properties) { 44 | ignorables = new ArrayList<>(); 45 | for (String property : properties) { 46 | ignorables.add(property); 47 | } 48 | this.type = clazz; 49 | } 50 | 51 | @Override 52 | public BeanDeserializerBuilder updateBuilder( 53 | DeserializationConfig config, BeanDescription beanDesc, 54 | BeanDeserializerBuilder builder) { 55 | if (!type.equals(beanDesc.getBeanClass())) { 56 | return builder; 57 | } 58 | 59 | for (String ignorable : ignorables) { 60 | builder.addIgnorable(ignorable); 61 | } 62 | return builder; 63 | } 64 | 65 | @Override 66 | public List updateProperties( 67 | DeserializationConfig config, BeanDescription beanDesc, 68 | List propDefs) { 69 | if (!type.equals(beanDesc.getBeanClass())) { 70 | return propDefs; 71 | } 72 | 73 | List newPropDefs = new ArrayList<>(); 74 | for (BeanPropertyDefinition propDef : propDefs) { 75 | if (!ignorables.contains(propDef.getName())) { 76 | newPropDefs.add(propDef); 77 | } 78 | } 79 | return newPropDefs; 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/table/RowDataKinesisDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink.table; 22 | 23 | import org.apache.flink.annotation.Internal; 24 | import org.apache.flink.api.common.serialization.DeserializationSchema; 25 | import org.apache.flink.api.common.typeinfo.TypeInformation; 26 | import org.apache.flink.table.data.RowData; 27 | import org.apache.flink.util.Preconditions; 28 | 29 | import software.amazon.kinesis.connectors.flink.serialization.KinesisDeserializationSchema; 30 | 31 | import java.io.IOException; 32 | 33 | /** 34 | * A {@link KinesisDeserializationSchema} adaptor for {@link RowData} records that delegates 35 | * physical data deserialization to an inner {@link DeserializationSchema} and appends requested 36 | * metadata to the end of the deserialized {@link RowData} record. 37 | */ 38 | @Internal 39 | public final class RowDataKinesisDeserializationSchema 40 | implements KinesisDeserializationSchema { 41 | 42 | private static final long serialVersionUID = 5551095193778230749L; 43 | 44 | /** A {@link DeserializationSchema} to deserialize the physical part of the row. */ 45 | private final DeserializationSchema physicalDeserializer; 46 | 47 | /** The type of the produced {@link RowData} records (physical data with appended metadata]. */ 48 | private final TypeInformation producedTypeInfo; 49 | 50 | public RowDataKinesisDeserializationSchema( 51 | DeserializationSchema physicalDeserializer, 52 | TypeInformation producedTypeInfo) { 53 | this.physicalDeserializer = Preconditions.checkNotNull(physicalDeserializer); 54 | this.producedTypeInfo = Preconditions.checkNotNull(producedTypeInfo); 55 | } 56 | 57 | @Override 58 | public void open(DeserializationSchema.InitializationContext context) throws Exception { 59 | physicalDeserializer.open(context); 60 | } 61 | 62 | @Override 63 | public RowData deserialize( 64 | byte[] recordValue, 65 | String partitionKey, 66 | String seqNum, 67 | long approxArrivalTimestamp, 68 | String stream, 69 | String shardId) throws IOException { 70 | 71 | return physicalDeserializer.deserialize(recordValue); 72 | } 73 | 74 | @Override 75 | public TypeInformation getProducedType() { 76 | return producedTypeInfo; 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/model/SentinelSequenceNumber.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import software.amazon.kinesis.connectors.flink.FlinkKinesisConsumer; 25 | import software.amazon.kinesis.connectors.flink.internals.KinesisDataFetcher; 26 | 27 | /** 28 | * Special flag values for sequence numbers in shards to indicate special positions. 29 | * The value is initially set by {@link FlinkKinesisConsumer} when {@link KinesisDataFetcher}s are created. 30 | * The KinesisDataFetchers will use this value to determine how to retrieve the starting shard iterator from AWS Kinesis. 31 | */ 32 | @Internal 33 | public enum SentinelSequenceNumber { 34 | 35 | /** Flag value for shard's sequence numbers to indicate that the 36 | * shard should start to be read from the latest incoming records. */ 37 | SENTINEL_LATEST_SEQUENCE_NUM(new SequenceNumber("LATEST_SEQUENCE_NUM")), 38 | 39 | /** Flag value for shard's sequence numbers to indicate that the shard should 40 | * start to be read from the earliest records that haven't expired yet. */ 41 | SENTINEL_EARLIEST_SEQUENCE_NUM(new SequenceNumber("EARLIEST_SEQUENCE_NUM")), 42 | 43 | /** Flag value for shard's sequence numbers to indicate that the shard should 44 | * start to be read from the specified timestamp. */ 45 | SENTINEL_AT_TIMESTAMP_SEQUENCE_NUM(new SequenceNumber("AT_TIMESTAMP_SEQUENCE_NUM")), 46 | 47 | /** Flag value to indicate that we have already read the last record of this shard 48 | * (Note: Kinesis shards that have been closed due to a split or merge will have an ending data record). */ 49 | SENTINEL_SHARD_ENDING_SEQUENCE_NUM(new SequenceNumber("SHARD_ENDING_SEQUENCE_NUM")); 50 | 51 | private SequenceNumber sentinel; 52 | 53 | SentinelSequenceNumber(SequenceNumber sentinel) { 54 | this.sentinel = sentinel; 55 | } 56 | 57 | public SequenceNumber get() { 58 | return sentinel; 59 | } 60 | 61 | /** 62 | * Returns {@code true} if the given {@link SequenceNumber} is a sentinel. 63 | */ 64 | public static boolean isSentinelSequenceNumber(SequenceNumber candidateSequenceNumber) { 65 | for (SentinelSequenceNumber sentinel : values()) { 66 | if (candidateSequenceNumber.equals(sentinel.get())) { 67 | return true; 68 | } 69 | } 70 | return false; 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/proxy/KinesisProxyV2Factory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.proxy; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.util.Preconditions; 24 | 25 | import com.amazonaws.ClientConfiguration; 26 | import com.amazonaws.ClientConfigurationFactory; 27 | import software.amazon.awssdk.http.async.SdkAsyncHttpClient; 28 | import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; 29 | import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; 30 | import software.amazon.kinesis.connectors.flink.internals.publisher.fanout.FanOutRecordPublisherConfiguration; 31 | import software.amazon.kinesis.connectors.flink.util.AWSUtil; 32 | import software.amazon.kinesis.connectors.flink.util.AwsV2Util; 33 | 34 | import java.util.Properties; 35 | 36 | import static java.util.Collections.emptyList; 37 | 38 | /** 39 | * Creates instances of {@link KinesisProxyV2}. 40 | */ 41 | @Internal 42 | public class KinesisProxyV2Factory { 43 | 44 | private static final FullJitterBackoff BACKOFF = new FullJitterBackoff(); 45 | 46 | /** 47 | * Uses the given properties to instantiate a new instance of {@link KinesisProxyV2}. 48 | * 49 | * @param configProps the properties used to parse configuration 50 | * @return the Kinesis proxy 51 | */ 52 | public static KinesisProxyV2Interface createKinesisProxyV2(final Properties configProps) { 53 | Preconditions.checkNotNull(configProps); 54 | 55 | final ClientConfiguration clientConfiguration = new ClientConfigurationFactory().getConfig(); 56 | populateDefaultValues(clientConfiguration); 57 | AWSUtil.setAwsClientConfigProperties(clientConfiguration, configProps); 58 | 59 | final SdkAsyncHttpClient httpClient = AwsV2Util.createHttpClient(clientConfiguration, NettyNioAsyncHttpClient.builder(), configProps); 60 | final FanOutRecordPublisherConfiguration configuration = new FanOutRecordPublisherConfiguration(configProps, emptyList()); 61 | final KinesisAsyncClient client = AwsV2Util.createKinesisAsyncClient(configProps, clientConfiguration, httpClient); 62 | 63 | return new KinesisProxyV2(client, httpClient, configuration, BACKOFF); 64 | } 65 | 66 | private static void populateDefaultValues(final ClientConfiguration clientConfiguration) { 67 | clientConfiguration.setUseTcpKeepAlive(true); 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/FlinkDynamoDBStreamsConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink; 21 | 22 | import org.apache.flink.api.common.functions.RuntimeContext; 23 | import org.apache.flink.api.common.serialization.DeserializationSchema; 24 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 25 | 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | import software.amazon.kinesis.connectors.flink.internals.DynamoDBStreamsDataFetcher; 29 | import software.amazon.kinesis.connectors.flink.internals.KinesisDataFetcher; 30 | import software.amazon.kinesis.connectors.flink.serialization.KinesisDeserializationSchema; 31 | 32 | import java.util.List; 33 | import java.util.Properties; 34 | 35 | /** 36 | * Consume events from DynamoDB streams. 37 | * 38 | * @param the type of data emitted 39 | */ 40 | public class FlinkDynamoDBStreamsConsumer extends FlinkKinesisConsumer { 41 | private static final Logger LOG = LoggerFactory.getLogger(FlinkDynamoDBStreamsConsumer.class); 42 | 43 | /** 44 | * Constructor of FlinkDynamoDBStreamsConsumer. 45 | * 46 | * @param stream stream to consume 47 | * @param deserializer deserialization schema 48 | * @param config config properties 49 | */ 50 | public FlinkDynamoDBStreamsConsumer( 51 | String stream, 52 | DeserializationSchema deserializer, 53 | Properties config) { 54 | super(stream, deserializer, config); 55 | } 56 | 57 | /** 58 | * Constructor of FlinkDynamodbStreamConsumer. 59 | * 60 | * @param streams list of streams to consume 61 | * @param deserializer deserialization schema 62 | * @param config config properties 63 | */ 64 | public FlinkDynamoDBStreamsConsumer( 65 | List streams, 66 | KinesisDeserializationSchema deserializer, 67 | Properties config) { 68 | super(streams, deserializer, config); 69 | } 70 | 71 | @Override 72 | protected KinesisDataFetcher createFetcher( 73 | List streams, 74 | SourceFunction.SourceContext sourceContext, 75 | RuntimeContext runtimeContext, 76 | Properties configProps, 77 | KinesisDeserializationSchema deserializationSchema) { 78 | return new DynamoDBStreamsDataFetcher( 79 | streams, 80 | sourceContext, 81 | runtimeContext, 82 | configProps, 83 | deserializationSchema, 84 | getShardAssigner()); 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/serialization/KinesisDeserializationSchemaWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.serialization; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.api.common.serialization.DeserializationSchema; 24 | import org.apache.flink.api.common.typeinfo.TypeInformation; 25 | import org.apache.flink.util.Collector; 26 | 27 | import java.io.IOException; 28 | 29 | /** 30 | * A simple wrapper for using the {@link DeserializationSchema} with the {@link KinesisDeserializationSchema} interface. 31 | * 32 | * @param The type created by the deserialization schema. 33 | */ 34 | @Internal 35 | public class KinesisDeserializationSchemaWrapper implements KinesisDeserializationSchema { 36 | private static final long serialVersionUID = 9143148962928375886L; 37 | 38 | private final DeserializationSchema deserializationSchema; 39 | 40 | public KinesisDeserializationSchemaWrapper(DeserializationSchema deserializationSchema) { 41 | try { 42 | Class deserilizationClass = deserializationSchema.getClass(); 43 | if (!deserilizationClass.getMethod("deserialize", byte[].class, Collector.class).isDefault()) { 44 | throw new IllegalArgumentException( 45 | "Kinesis consumer does not support DeserializationSchema that implements " + 46 | "deserialization with a Collector. Unsupported DeserializationSchema: " + 47 | deserilizationClass.getName()); 48 | } 49 | } catch (NoSuchMethodException e) { 50 | // swallow the exception 51 | } 52 | this.deserializationSchema = deserializationSchema; 53 | } 54 | 55 | @Override 56 | public void open(DeserializationSchema.InitializationContext context) throws Exception { 57 | this.deserializationSchema.open(context); 58 | } 59 | 60 | @Override 61 | public T deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) 62 | throws IOException { 63 | return deserializationSchema.deserialize(recordValue); 64 | } 65 | 66 | /* 67 | FLINK-4194 68 | 69 | @Override 70 | public boolean isEndOfStream(T nextElement) { 71 | return deserializationSchema.isEndOfStream(nextElement); 72 | } */ 73 | 74 | @Override 75 | public TypeInformation getProducedType() { 76 | return deserializationSchema.getProducedType(); 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/TestRuntimeContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | import org.apache.flink.api.common.ExecutionConfig; 23 | import org.apache.flink.metrics.MetricGroup; 24 | import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; 25 | import org.apache.flink.runtime.jobgraph.OperatorID; 26 | import org.apache.flink.runtime.memory.MemoryManager; 27 | import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; 28 | import org.apache.flink.streaming.api.operators.AbstractStreamOperator; 29 | import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; 30 | 31 | import java.util.Collections; 32 | 33 | /** 34 | * A mock {@link StreamingRuntimeContext} for testing. 35 | */ 36 | public class TestRuntimeContext extends StreamingRuntimeContext { 37 | 38 | private final boolean isCheckpointingEnabled; 39 | 40 | private final int numParallelSubtasks; 41 | private final int subtaskIndex; 42 | 43 | public TestRuntimeContext( 44 | boolean isCheckpointingEnabled, 45 | int numParallelSubtasks, 46 | int subtaskIndex) { 47 | 48 | super( 49 | new TestStreamOperator(), 50 | new MockEnvironmentBuilder() 51 | .setTaskName("mockTask") 52 | .setManagedMemorySize(4 * MemoryManager.DEFAULT_PAGE_SIZE) 53 | .build(), 54 | Collections.emptyMap()); 55 | 56 | this.isCheckpointingEnabled = isCheckpointingEnabled; 57 | this.numParallelSubtasks = numParallelSubtasks; 58 | this.subtaskIndex = subtaskIndex; 59 | } 60 | 61 | @Override 62 | public MetricGroup getMetricGroup() { 63 | return new UnregisteredMetricsGroup(); 64 | } 65 | 66 | @Override 67 | public boolean isCheckpointingEnabled() { 68 | return isCheckpointingEnabled; 69 | } 70 | 71 | @Override 72 | public int getIndexOfThisSubtask() { 73 | return subtaskIndex; 74 | } 75 | 76 | @Override 77 | public int getNumberOfParallelSubtasks() { 78 | return numParallelSubtasks; 79 | } 80 | 81 | private static class TestStreamOperator extends AbstractStreamOperator { 82 | 83 | private static final long serialVersionUID = -2547912462252989589L; 84 | 85 | @Override 86 | public ExecutionConfig getExecutionConfig() { 87 | return new ExecutionConfig(); 88 | } 89 | 90 | @Override 91 | public OperatorID getOperatorID() { 92 | return new OperatorID(42, 44); 93 | } 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/internals/DynamoDBStreamsDataFetcherTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"). 5 | * You may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package software.amazon.kinesis.connectors.flink.internals; 18 | 19 | import org.apache.flink.api.common.functions.RuntimeContext; 20 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 21 | import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; 22 | 23 | import org.junit.Test; 24 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 25 | import software.amazon.kinesis.connectors.flink.proxy.KinesisProxyInterface; 26 | import software.amazon.kinesis.connectors.flink.serialization.KinesisDeserializationSchemaWrapper; 27 | import software.amazon.kinesis.connectors.flink.testutils.TestSourceContext; 28 | import software.amazon.kinesis.connectors.flink.testutils.TestUtils; 29 | 30 | import java.util.Properties; 31 | 32 | import static com.amazonaws.services.kinesis.model.ShardIteratorType.LATEST; 33 | import static java.util.Collections.singletonList; 34 | import static org.mockito.Mockito.mock; 35 | import static org.mockito.Mockito.verify; 36 | import static software.amazon.kinesis.connectors.flink.internals.KinesisDataFetcher.DEFAULT_SHARD_ASSIGNER; 37 | import static software.amazon.kinesis.connectors.flink.internals.ShardConsumerTestUtils.createFakeShardConsumerMetricGroup; 38 | import static software.amazon.kinesis.connectors.flink.model.SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM; 39 | 40 | /** 41 | * Tests for {@link DynamoDBStreamsDataFetcher}. 42 | */ 43 | public class DynamoDBStreamsDataFetcherTest { 44 | 45 | @Test 46 | public void testCreateRecordPublisherRespectsShardIteratorTypeLatest() throws Exception { 47 | RuntimeContext runtimeContext = TestUtils.getMockedRuntimeContext(1, 0); 48 | KinesisProxyInterface kinesis = mock(KinesisProxyInterface.class); 49 | 50 | DynamoDBStreamsDataFetcher fetcher = new DynamoDBStreamsDataFetcher<>( 51 | singletonList("fakeStream"), 52 | new TestSourceContext<>(), 53 | runtimeContext, 54 | TestUtils.getStandardProperties(), 55 | new KinesisDeserializationSchemaWrapper<>(new SimpleStringSchema()), 56 | DEFAULT_SHARD_ASSIGNER, 57 | config -> kinesis); 58 | 59 | StreamShardHandle dummyStreamShardHandle = TestUtils.createDummyStreamShardHandle("dummy-stream", "0"); 60 | 61 | fetcher.createRecordPublisher( 62 | SENTINEL_LATEST_SEQUENCE_NUM.get(), 63 | new Properties(), 64 | createFakeShardConsumerMetricGroup((OperatorMetricGroup) runtimeContext.getMetricGroup()), 65 | dummyStreamShardHandle); 66 | 67 | verify(kinesis).getShardIterator(dummyStreamShardHandle, LATEST.toString(), null); 68 | } 69 | 70 | } 71 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/internals/publisher/polling/PollingRecordPublisherFactoryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher.polling; 21 | 22 | import org.junit.Test; 23 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisher; 24 | import software.amazon.kinesis.connectors.flink.model.StartingPosition; 25 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 26 | import software.amazon.kinesis.connectors.flink.proxy.KinesisProxy; 27 | 28 | import java.util.Properties; 29 | 30 | import static org.junit.Assert.assertFalse; 31 | import static org.junit.Assert.assertTrue; 32 | import static org.mockito.Mockito.mock; 33 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS; 34 | import static software.amazon.kinesis.connectors.flink.internals.ShardConsumerTestUtils.createFakeShardConsumerMetricGroup; 35 | import static software.amazon.kinesis.connectors.flink.model.SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM; 36 | 37 | /** 38 | * Tests for {@link PollingRecordPublisherFactory}. 39 | */ 40 | public class PollingRecordPublisherFactoryTest { 41 | 42 | private final PollingRecordPublisherFactory factory = new PollingRecordPublisherFactory(props -> mock(KinesisProxy.class)); 43 | 44 | @Test 45 | public void testBuildPollingRecordPublisher() throws Exception { 46 | RecordPublisher recordPublisher = factory.create( 47 | StartingPosition.restartFromSequenceNumber(SENTINEL_LATEST_SEQUENCE_NUM.get()), 48 | new Properties(), 49 | createFakeShardConsumerMetricGroup(), 50 | mock(StreamShardHandle.class)); 51 | 52 | assertTrue(recordPublisher instanceof PollingRecordPublisher); 53 | assertFalse(recordPublisher instanceof AdaptivePollingRecordPublisher); 54 | } 55 | 56 | @Test 57 | public void testBuildAdaptivePollingRecordPublisher() throws Exception { 58 | Properties properties = new Properties(); 59 | properties.setProperty(SHARD_USE_ADAPTIVE_READS, "true"); 60 | 61 | RecordPublisher recordPublisher = factory.create( 62 | StartingPosition.restartFromSequenceNumber(SENTINEL_LATEST_SEQUENCE_NUM.get()), 63 | properties, 64 | createFakeShardConsumerMetricGroup(), 65 | mock(StreamShardHandle.class)); 66 | 67 | assertTrue(recordPublisher instanceof PollingRecordPublisher); 68 | assertTrue(recordPublisher instanceof AdaptivePollingRecordPublisher); 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/internals/publisher/polling/PollingRecordPublisherConfigurationTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher.polling; 21 | 22 | import org.junit.Test; 23 | 24 | import java.util.Properties; 25 | 26 | import static org.junit.Assert.assertEquals; 27 | import static org.junit.Assert.assertFalse; 28 | import static org.junit.Assert.assertTrue; 29 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.SHARD_GETRECORDS_INTERVAL_MILLIS; 30 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.SHARD_GETRECORDS_MAX; 31 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.SHARD_USE_ADAPTIVE_READS; 32 | 33 | /** 34 | * Tests for {@link PollingRecordPublisherConfiguration}. 35 | */ 36 | public class PollingRecordPublisherConfigurationTest { 37 | 38 | @Test 39 | public void testDefaults() { 40 | PollingRecordPublisherConfiguration configuration = new PollingRecordPublisherConfiguration(new Properties()); 41 | assertEquals(configuration.getFetchIntervalMillis(), 200); 42 | assertEquals(configuration.getMaxNumberOfRecordsPerFetch(), 10000); 43 | assertFalse(configuration.isAdaptiveReads()); 44 | } 45 | 46 | @Test 47 | public void testGetFetchIntervalMillis() { 48 | Properties properties = properties(SHARD_GETRECORDS_INTERVAL_MILLIS, "1"); 49 | PollingRecordPublisherConfiguration configuration = new PollingRecordPublisherConfiguration(properties); 50 | 51 | assertEquals(configuration.getFetchIntervalMillis(), 1); 52 | } 53 | 54 | @Test 55 | public void testGetMaxNumberOfRecordsPerFetch() { 56 | Properties properties = properties(SHARD_GETRECORDS_MAX, "2"); 57 | PollingRecordPublisherConfiguration configuration = new PollingRecordPublisherConfiguration(properties); 58 | 59 | assertEquals(configuration.getMaxNumberOfRecordsPerFetch(), 2); 60 | } 61 | 62 | @Test 63 | public void testIsAdaptiveReads() { 64 | Properties properties = properties(SHARD_USE_ADAPTIVE_READS, "true"); 65 | PollingRecordPublisherConfiguration configuration = new PollingRecordPublisherConfiguration(properties); 66 | 67 | assertTrue(configuration.isAdaptiveReads()); 68 | } 69 | 70 | private Properties properties(final String key, final String value) { 71 | final Properties properties = new Properties(); 72 | properties.setProperty(key, value); 73 | return properties; 74 | } 75 | 76 | } 77 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/metrics/PollingRecordPublisherMetricsReporter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink.metrics; 22 | 23 | import org.apache.flink.annotation.Internal; 24 | import org.apache.flink.metrics.MetricGroup; 25 | 26 | import software.amazon.kinesis.connectors.flink.internals.publisher.polling.PollingRecordPublisher; 27 | 28 | /** 29 | * A container for {@link PollingRecordPublisher}s to report metric values. 30 | */ 31 | @Internal 32 | public class PollingRecordPublisherMetricsReporter { 33 | 34 | private volatile double loopFrequencyHz = 0.0; 35 | private volatile double bytesPerRead = 0.0; 36 | private volatile long runLoopTimeNanos = 0L; 37 | private volatile long sleepTimeMillis = 0L; 38 | private volatile int maxNumberOfRecordsPerFetch = 0; 39 | 40 | public PollingRecordPublisherMetricsReporter(final MetricGroup metricGroup) { 41 | metricGroup.gauge(KinesisConsumerMetricConstants.MAX_RECORDS_PER_FETCH, this::getMaxNumberOfRecordsPerFetch); 42 | metricGroup.gauge(KinesisConsumerMetricConstants.BYTES_PER_READ, this::getBytesPerRead); 43 | metricGroup.gauge(KinesisConsumerMetricConstants.RUNTIME_LOOP_NANOS, this::getRunLoopTimeNanos); 44 | metricGroup.gauge(KinesisConsumerMetricConstants.LOOP_FREQUENCY_HZ, this::getLoopFrequencyHz); 45 | metricGroup.gauge(KinesisConsumerMetricConstants.SLEEP_TIME_MILLIS, this::getSleepTimeMillis); 46 | } 47 | 48 | public double getLoopFrequencyHz() { 49 | return loopFrequencyHz; 50 | } 51 | 52 | public void setLoopFrequencyHz(double loopFrequencyHz) { 53 | this.loopFrequencyHz = loopFrequencyHz; 54 | } 55 | 56 | public double getBytesPerRead() { 57 | return bytesPerRead; 58 | } 59 | 60 | public void setBytesPerRead(double bytesPerRead) { 61 | this.bytesPerRead = bytesPerRead; 62 | } 63 | 64 | public long getRunLoopTimeNanos() { 65 | return runLoopTimeNanos; 66 | } 67 | 68 | public void setRunLoopTimeNanos(long runLoopTimeNanos) { 69 | this.runLoopTimeNanos = runLoopTimeNanos; 70 | } 71 | 72 | public long getSleepTimeMillis() { 73 | return sleepTimeMillis; 74 | } 75 | 76 | public void setSleepTimeMillis(long sleepTimeMillis) { 77 | this.sleepTimeMillis = sleepTimeMillis; 78 | } 79 | 80 | public int getMaxNumberOfRecordsPerFetch() { 81 | return maxNumberOfRecordsPerFetch; 82 | } 83 | 84 | public void setMaxNumberOfRecordsPerFetch(int maxNumberOfRecordsPerFetch) { 85 | this.maxNumberOfRecordsPerFetch = maxNumberOfRecordsPerFetch; 86 | } 87 | 88 | } 89 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/examples/ProduceIntoKinesis.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.examples; 21 | 22 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 23 | import org.apache.flink.api.java.utils.ParameterTool; 24 | import org.apache.flink.streaming.api.datastream.DataStream; 25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 26 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 27 | 28 | import org.apache.commons.lang3.RandomStringUtils; 29 | import software.amazon.kinesis.connectors.flink.FlinkKinesisProducer; 30 | import software.amazon.kinesis.connectors.flink.config.AWSConfigConstants; 31 | 32 | import java.util.Properties; 33 | 34 | /** 35 | * This is an example on how to produce data into Kinesis. 36 | */ 37 | public class ProduceIntoKinesis { 38 | 39 | public static void main(String[] args) throws Exception { 40 | ParameterTool pt = ParameterTool.fromArgs(args); 41 | 42 | StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); 43 | see.setParallelism(1); 44 | 45 | DataStream simpleStringStream = see.addSource(new EventsGenerator()); 46 | 47 | Properties kinesisProducerConfig = new Properties(); 48 | kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region")); 49 | kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); 50 | kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); 51 | 52 | FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>( 53 | new SimpleStringSchema(), kinesisProducerConfig); 54 | 55 | kinesis.setFailOnError(true); 56 | kinesis.setDefaultStream("flink-test"); 57 | kinesis.setDefaultPartition("0"); 58 | 59 | simpleStringStream.addSink(kinesis); 60 | 61 | see.execute(); 62 | } 63 | 64 | /** 65 | * Data generator that creates strings starting with a sequence number followed by a dash and 12 random characters. 66 | */ 67 | public static class EventsGenerator implements SourceFunction { 68 | private boolean running = true; 69 | 70 | @Override 71 | public void run(SourceContext ctx) throws Exception { 72 | long seq = 0; 73 | while (running) { 74 | Thread.sleep(10); 75 | ctx.collect((seq++) + "-" + RandomStringUtils.randomAlphabetic(12)); 76 | } 77 | } 78 | 79 | @Override 80 | public void cancel() { 81 | running = false; 82 | } 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/metrics/ShardConsumerMetricsReporterTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.metrics; 21 | 22 | import org.apache.flink.metrics.MetricGroup; 23 | import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; 24 | 25 | import org.junit.Before; 26 | import org.junit.Test; 27 | import org.mockito.InjectMocks; 28 | import org.mockito.Mock; 29 | import org.mockito.MockitoAnnotations; 30 | import software.amazon.kinesis.connectors.flink.internals.ShardConsumerTestUtils; 31 | 32 | import static org.junit.Assert.assertEquals; 33 | import static org.junit.Assert.assertTrue; 34 | import static org.mockito.ArgumentMatchers.any; 35 | import static org.mockito.ArgumentMatchers.eq; 36 | import static org.mockito.Mockito.verify; 37 | 38 | /** 39 | * Tests for {@link ShardConsumerMetricsReporter}. 40 | */ 41 | public class ShardConsumerMetricsReporterTest { 42 | 43 | @InjectMocks 44 | private ShardConsumerMetricsReporter metricsReporter; 45 | 46 | @Mock 47 | private MetricGroup metricGroup; 48 | 49 | @Before 50 | public void setUp() { 51 | MockitoAnnotations.initMocks(this); 52 | } 53 | 54 | @Test 55 | public void testMetricIdentifiers() { 56 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.AVG_RECORD_SIZE_BYTES), any()); 57 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.MILLIS_BEHIND_LATEST_GAUGE), any()); 58 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.NUM_AGGREGATED_RECORDS_PER_FETCH), any()); 59 | verify(metricGroup).gauge(eq(KinesisConsumerMetricConstants.NUM_DEAGGREGATED_RECORDS_PER_FETCH), any()); 60 | } 61 | 62 | @Test 63 | public void testGettersAndSetters() { 64 | metricsReporter.setAverageRecordSizeBytes(1); 65 | metricsReporter.setMillisBehindLatest(2); 66 | metricsReporter.setNumberOfAggregatedRecords(3); 67 | metricsReporter.setNumberOfDeaggregatedRecords(4); 68 | 69 | assertEquals(1, metricsReporter.getAverageRecordSizeBytes()); 70 | assertEquals(2, metricsReporter.getMillisBehindLatest()); 71 | assertEquals(3, metricsReporter.getNumberOfAggregatedRecords()); 72 | assertEquals(4, metricsReporter.getNumberOfDeaggregatedRecords()); 73 | } 74 | 75 | @Test 76 | public void testUnregister() { 77 | AbstractMetricGroup metricGroup = ShardConsumerTestUtils 78 | .createFakeShardConsumerMetricGroup(); 79 | ShardConsumerMetricsReporter metricsReporter = new ShardConsumerMetricsReporter(metricGroup); 80 | 81 | metricsReporter.unregister(); 82 | 83 | assertTrue(metricGroup.isClosed()); 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /CONTRIBUTING.md: -------------------------------------------------------------------------------- 1 | # Contributing Guidelines 2 | 3 | Thank you for your interest in contributing to our project. Whether it's a bug report, new feature, correction, or additional 4 | documentation, we greatly value feedback and contributions from our community. 5 | 6 | Please read through this document before submitting any issues or pull requests to ensure we have all the necessary 7 | information to effectively respond to your bug report or contribution. 8 | 9 | 10 | ## Reporting Bugs/Feature Requests 11 | 12 | We welcome you to use the GitHub issue tracker to report bugs or suggest features. 13 | 14 | When filing an issue, please check existing open, or recently closed, issues to make sure somebody else hasn't already 15 | reported the issue. Please try to include as much information as you can. Details like these are incredibly useful: 16 | 17 | * A reproducible test case or series of steps 18 | * The version of our code being used 19 | * Any modifications you've made relevant to the bug 20 | * Anything unusual about your environment or deployment 21 | 22 | 23 | ## Contributing via Pull Requests 24 | Contributions via pull requests are much appreciated. Before sending us a pull request, please ensure that: 25 | 26 | 1. You are working against the latest source on the *master* branch. 27 | 2. You check existing open, and recently merged, pull requests to make sure someone else hasn't addressed the problem already. 28 | 3. You open an issue to discuss any significant work - we would hate for your time to be wasted. 29 | 30 | To send us a pull request, please: 31 | 32 | 1. Fork the repository. 33 | 2. Modify the source; please focus on the specific change you are contributing. If you also reformat all the code, it will be hard for us to focus on your change. 34 | 3. Ensure local tests pass. 35 | 4. Commit to your fork using clear commit messages. 36 | 5. Send us a pull request, answering any default questions in the pull request interface. 37 | 6. Pay attention to any automated CI failures reported in the pull request, and stay involved in the conversation. 38 | 39 | GitHub provides additional document on [forking a repository](https://help.github.com/articles/fork-a-repo/) and 40 | [creating a pull request](https://help.github.com/articles/creating-a-pull-request/). 41 | 42 | 43 | ## Finding contributions to work on 44 | Looking at the existing issues is a great way to find something to contribute on. As our projects, by default, use the default GitHub issue labels (enhancement/bug/duplicate/help wanted/invalid/question/wontfix), looking at any 'help wanted' issues is a great place to start. 45 | 46 | 47 | ## Code of Conduct 48 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct). 49 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact 50 | opensource-codeofconduct@amazon.com with any additional questions or comments. 51 | 52 | 53 | ## Security issue notifications 54 | If you discover a potential security issue in this project we ask that you notify AWS/Amazon Security via our [vulnerability reporting page](http://aws.amazon.com/security/vulnerability-reporting/). Please do **not** create a public github issue. 55 | 56 | 57 | ## Licensing 58 | 59 | See the [LICENSE](LICENSE) file for our project's licensing. We will ask you to confirm the licensing of your contribution. 60 | 61 | We may ask you to sign a [Contributor License Agreement (CLA)](http://en.wikipedia.org/wiki/Contributor_License_Agreement) for larger changes. 62 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/metrics/ShardConsumerMetricsReporter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package software.amazon.kinesis.connectors.flink.metrics; 22 | 23 | import org.apache.flink.annotation.Internal; 24 | import org.apache.flink.metrics.MetricGroup; 25 | import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup; 26 | 27 | import software.amazon.kinesis.connectors.flink.internals.ShardConsumer; 28 | 29 | /** 30 | * A container for {@link ShardConsumer}s to report metric values. 31 | */ 32 | @Internal 33 | public class ShardConsumerMetricsReporter { 34 | 35 | private final MetricGroup metricGroup; 36 | 37 | private volatile long millisBehindLatest = -1; 38 | private volatile long averageRecordSizeBytes = 0L; 39 | private volatile int numberOfAggregatedRecords = 0; 40 | private volatile int numberOfDeaggregatedRecords = 0; 41 | 42 | public ShardConsumerMetricsReporter(final MetricGroup metricGroup) { 43 | this.metricGroup = metricGroup; 44 | metricGroup.gauge(KinesisConsumerMetricConstants.MILLIS_BEHIND_LATEST_GAUGE, this::getMillisBehindLatest); 45 | metricGroup.gauge(KinesisConsumerMetricConstants.NUM_AGGREGATED_RECORDS_PER_FETCH, this::getNumberOfAggregatedRecords); 46 | metricGroup.gauge(KinesisConsumerMetricConstants.NUM_DEAGGREGATED_RECORDS_PER_FETCH, this::getNumberOfDeaggregatedRecords); 47 | metricGroup.gauge(KinesisConsumerMetricConstants.AVG_RECORD_SIZE_BYTES, this::getAverageRecordSizeBytes); 48 | } 49 | 50 | public long getMillisBehindLatest() { 51 | return millisBehindLatest; 52 | } 53 | 54 | public void setMillisBehindLatest(long millisBehindLatest) { 55 | this.millisBehindLatest = millisBehindLatest; 56 | } 57 | 58 | public long getAverageRecordSizeBytes() { 59 | return averageRecordSizeBytes; 60 | } 61 | 62 | public void setAverageRecordSizeBytes(long averageRecordSizeBytes) { 63 | this.averageRecordSizeBytes = averageRecordSizeBytes; 64 | } 65 | 66 | public int getNumberOfAggregatedRecords() { 67 | return numberOfAggregatedRecords; 68 | } 69 | 70 | public void setNumberOfAggregatedRecords(int numberOfAggregatedRecords) { 71 | this.numberOfAggregatedRecords = numberOfAggregatedRecords; 72 | } 73 | 74 | public int getNumberOfDeaggregatedRecords() { 75 | return numberOfDeaggregatedRecords; 76 | } 77 | 78 | public void setNumberOfDeaggregatedRecords(int numberOfDeaggregatedRecords) { 79 | this.numberOfDeaggregatedRecords = numberOfDeaggregatedRecords; 80 | } 81 | 82 | public void unregister() { 83 | if (this.metricGroup instanceof AbstractMetricGroup) { 84 | ((AbstractMetricGroup) this.metricGroup).close(); 85 | } 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/publisher/RecordBatch.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.util.Preconditions; 24 | 25 | import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; 26 | import com.amazonaws.services.kinesis.model.Record; 27 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 28 | 29 | import javax.annotation.Nullable; 30 | 31 | import java.math.BigInteger; 32 | import java.util.List; 33 | 34 | /** 35 | * A batch of UserRecords received from Kinesis. 36 | * Input records are de-aggregated using KCL 1.x library. 37 | * It is expected that AWS SDK v2.x messages are converted to KCL 1.x {@link UserRecord}. 38 | */ 39 | @Internal 40 | public class RecordBatch { 41 | 42 | private final int aggregatedRecordSize; 43 | 44 | private final List deaggregatedRecords; 45 | 46 | private final long totalSizeInBytes; 47 | 48 | private final Long millisBehindLatest; 49 | 50 | public RecordBatch( 51 | final List records, 52 | final StreamShardHandle subscribedShard, 53 | @Nullable final Long millisBehindLatest) { 54 | Preconditions.checkNotNull(subscribedShard); 55 | this.aggregatedRecordSize = Preconditions.checkNotNull(records).size(); 56 | this.deaggregatedRecords = deaggregateRecords(records, subscribedShard); 57 | this.totalSizeInBytes = this.deaggregatedRecords.stream().mapToInt(r -> r.getData().remaining()).sum(); 58 | this.millisBehindLatest = millisBehindLatest; 59 | } 60 | 61 | public int getAggregatedRecordSize() { 62 | return aggregatedRecordSize; 63 | } 64 | 65 | public int getDeaggregatedRecordSize() { 66 | return deaggregatedRecords.size(); 67 | } 68 | 69 | public List getDeaggregatedRecords() { 70 | return deaggregatedRecords; 71 | } 72 | 73 | public long getTotalSizeInBytes() { 74 | return totalSizeInBytes; 75 | } 76 | 77 | public long getAverageRecordSizeBytes() { 78 | return deaggregatedRecords.isEmpty() ? 0 : getTotalSizeInBytes() / getDeaggregatedRecordSize(); 79 | } 80 | 81 | @Nullable 82 | public Long getMillisBehindLatest() { 83 | return millisBehindLatest; 84 | } 85 | 86 | private List deaggregateRecords(final List records, final StreamShardHandle subscribedShard) { 87 | BigInteger start = new BigInteger(subscribedShard.getShard().getHashKeyRange().getStartingHashKey()); 88 | BigInteger end = new BigInteger(subscribedShard.getShard().getHashKeyRange().getEndingHashKey()); 89 | 90 | return UserRecord.deaggregate(records, start, end); 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/model/KinesisStreamShardState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.util.Preconditions; 24 | 25 | import com.amazonaws.services.kinesis.model.Shard; 26 | 27 | /** 28 | * A wrapper class that bundles a {@link StreamShardHandle} with its last processed sequence number. 29 | */ 30 | @Internal 31 | public class KinesisStreamShardState { 32 | 33 | /** A handle object that wraps the actual {@link Shard} instance and stream name. */ 34 | private StreamShardHandle streamShardHandle; 35 | 36 | /** The checkpointed state for each Kinesis stream shard. */ 37 | private StreamShardMetadata streamShardMetadata; 38 | private SequenceNumber lastProcessedSequenceNum; 39 | 40 | public KinesisStreamShardState( 41 | StreamShardMetadata streamShardMetadata, 42 | StreamShardHandle streamShardHandle, 43 | SequenceNumber lastProcessedSequenceNum) { 44 | 45 | this.streamShardMetadata = Preconditions.checkNotNull(streamShardMetadata); 46 | this.streamShardHandle = Preconditions.checkNotNull(streamShardHandle); 47 | this.lastProcessedSequenceNum = Preconditions.checkNotNull(lastProcessedSequenceNum); 48 | } 49 | 50 | public StreamShardMetadata getStreamShardMetadata() { 51 | return this.streamShardMetadata; 52 | } 53 | 54 | public StreamShardHandle getStreamShardHandle() { 55 | return this.streamShardHandle; 56 | } 57 | 58 | public SequenceNumber getLastProcessedSequenceNum() { 59 | return this.lastProcessedSequenceNum; 60 | } 61 | 62 | public void setLastProcessedSequenceNum(SequenceNumber update) { 63 | this.lastProcessedSequenceNum = update; 64 | } 65 | 66 | @Override 67 | public String toString() { 68 | return "KinesisStreamShardState{" + 69 | "streamShardMetadata='" + streamShardMetadata.toString() + "'" + 70 | ", streamShardHandle='" + streamShardHandle.toString() + "'" + 71 | ", lastProcessedSequenceNumber='" + lastProcessedSequenceNum.toString() + "'}"; 72 | } 73 | 74 | @Override 75 | public boolean equals(Object obj) { 76 | if (!(obj instanceof KinesisStreamShardState)) { 77 | return false; 78 | } 79 | 80 | if (obj == this) { 81 | return true; 82 | } 83 | 84 | KinesisStreamShardState other = (KinesisStreamShardState) obj; 85 | 86 | return streamShardMetadata.equals(other.getStreamShardMetadata()) && 87 | streamShardHandle.equals(other.getStreamShardHandle()) && 88 | lastProcessedSequenceNum.equals(other.getLastProcessedSequenceNum()); 89 | } 90 | 91 | @Override 92 | public int hashCode() { 93 | return 37 * (streamShardMetadata.hashCode() + streamShardHandle.hashCode() + lastProcessedSequenceNum.hashCode()); 94 | } 95 | } 96 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/internals/publisher/RecordBatchTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher; 21 | 22 | import org.apache.flink.configuration.ConfigConstants; 23 | 24 | import com.amazonaws.services.kinesis.model.Record; 25 | import org.apache.commons.lang3.RandomStringUtils; 26 | import org.junit.Test; 27 | import software.amazon.kinesis.connectors.flink.testutils.TestUtils; 28 | 29 | import java.nio.ByteBuffer; 30 | import java.util.Arrays; 31 | import java.util.List; 32 | import java.util.concurrent.atomic.AtomicInteger; 33 | 34 | import static java.util.Collections.emptyList; 35 | import static java.util.Collections.singletonList; 36 | import static org.junit.Assert.assertEquals; 37 | import static software.amazon.kinesis.connectors.flink.testutils.TestUtils.createDummyStreamShardHandle; 38 | 39 | /** 40 | * Tests for {@link RecordBatch}. 41 | */ 42 | public class RecordBatchTest { 43 | 44 | @Test 45 | public void testDeaggregateRecordsPassThrough() { 46 | RecordBatch result = new RecordBatch(Arrays.asList( 47 | record("1"), 48 | record("2"), 49 | record("3"), 50 | record("4") 51 | ), createDummyStreamShardHandle(), 100L); 52 | 53 | assertEquals(4, result.getAggregatedRecordSize()); 54 | assertEquals(4, result.getDeaggregatedRecordSize()); 55 | assertEquals(128, result.getTotalSizeInBytes()); 56 | assertEquals(32, result.getAverageRecordSizeBytes()); 57 | } 58 | 59 | @Test 60 | public void testDeaggregateRecordsWithAggregatedRecords() { 61 | final List records = TestUtils.createAggregatedRecordBatch(5, 5, new AtomicInteger()); 62 | RecordBatch result = new RecordBatch(records, createDummyStreamShardHandle(), 100L); 63 | 64 | assertEquals(5, result.getAggregatedRecordSize()); 65 | assertEquals(25, result.getDeaggregatedRecordSize()); 66 | assertEquals(25 * 1024, result.getTotalSizeInBytes()); 67 | assertEquals(1024, result.getAverageRecordSizeBytes()); 68 | } 69 | 70 | @Test 71 | public void testGetAverageRecordSizeBytesEmptyList() { 72 | RecordBatch result = new RecordBatch(emptyList(), createDummyStreamShardHandle(), 100L); 73 | 74 | assertEquals(0, result.getAggregatedRecordSize()); 75 | assertEquals(0, result.getDeaggregatedRecordSize()); 76 | assertEquals(0, result.getAverageRecordSizeBytes()); 77 | } 78 | 79 | @Test 80 | public void testGetMillisBehindLatest() { 81 | RecordBatch result = new RecordBatch(singletonList(record("1")), createDummyStreamShardHandle(), 100L); 82 | 83 | assertEquals(Long.valueOf(100), result.getMillisBehindLatest()); 84 | } 85 | 86 | private Record record(final String sequenceNumber) { 87 | byte[] data = RandomStringUtils.randomAlphabetic(32) 88 | .getBytes(ConfigConstants.DEFAULT_CHARSET); 89 | 90 | return new Record() 91 | .withData(ByteBuffer.wrap(data)) 92 | .withPartitionKey("pk") 93 | .withSequenceNumber(sequenceNumber); 94 | } 95 | 96 | } 97 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/serialization/KinesisDeserializationSchema.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.serialization; 21 | 22 | import org.apache.flink.annotation.PublicEvolving; 23 | import org.apache.flink.api.common.serialization.DeserializationSchema; 24 | import org.apache.flink.api.java.typeutils.ResultTypeQueryable; 25 | 26 | import java.io.IOException; 27 | import java.io.Serializable; 28 | 29 | /** 30 | * This is a deserialization schema specific for the Flink Kinesis Consumer. Different from the 31 | * basic {@link DeserializationSchema}, this schema offers additional Kinesis-specific information 32 | * about the record that may be useful to the user application. 33 | * 34 | * @param The type created by the keyed deserialization schema. 35 | */ 36 | @PublicEvolving 37 | public interface KinesisDeserializationSchema extends Serializable, ResultTypeQueryable { 38 | 39 | /** 40 | * Initialization method for the schema. It is called before the actual working methods 41 | * {@link #deserialize} and thus suitable for one time setup work. 42 | * 43 | *

The provided {@link DeserializationSchema.InitializationContext} can be used to access additional features such as e.g. 44 | * registering user metrics. 45 | * 46 | * @param context Contextual information that can be used during initialization. 47 | */ 48 | default void open(DeserializationSchema.InitializationContext context) throws Exception { 49 | } 50 | 51 | /** 52 | * Deserializes a Kinesis record's bytes. If the record cannot be deserialized, {@code null} 53 | * may be returned. This informs the Flink Kinesis Consumer to process the Kinesis record 54 | * without producing any output for it, i.e. effectively "skipping" the record. 55 | * 56 | * @param recordValue the record's value as a byte array 57 | * @param partitionKey the record's partition key at the time of writing 58 | * @param seqNum the sequence number of this record in the Kinesis shard 59 | * @param approxArrivalTimestamp the server-side timestamp of when Kinesis received and stored the record 60 | * @param stream the name of the Kinesis stream that this record was sent to 61 | * @param shardId The identifier of the shard the record was sent to 62 | * 63 | * @return the deserialized message as an Java object ({@code null} if the message cannot be deserialized). 64 | * @throws IOException 65 | */ 66 | T deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException; 67 | 68 | /** 69 | * Method to decide whether the element signals the end of the stream. If 70 | * true is returned the element won't be emitted. 71 | * 72 | * @param nextElement the element to test for the end-of-stream signal 73 | * @return true if the element signals end of stream, false otherwise 74 | */ 75 | // TODO FLINK-4194 ADD SUPPORT FOR boolean isEndOfStream(T nextElement); 76 | } 77 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/util/JobManagerWatermarkTrackerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.util; 21 | 22 | import org.apache.flink.configuration.Configuration; 23 | import org.apache.flink.configuration.RestOptions; 24 | import org.apache.flink.runtime.minicluster.MiniCluster; 25 | import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; 26 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 27 | import org.apache.flink.streaming.api.functions.sink.SinkFunction; 28 | import org.apache.flink.streaming.api.functions.source.RichSourceFunction; 29 | 30 | import org.junit.AfterClass; 31 | import org.junit.Assert; 32 | import org.junit.BeforeClass; 33 | import org.junit.Test; 34 | 35 | /** Test for {@link JobManagerWatermarkTracker}. */ 36 | public class JobManagerWatermarkTrackerTest { 37 | 38 | private static MiniCluster flink; 39 | 40 | @BeforeClass 41 | public static void setUp() throws Exception { 42 | final Configuration config = new Configuration(); 43 | config.setInteger(RestOptions.PORT, 0); 44 | 45 | final MiniClusterConfiguration miniClusterConfiguration = new MiniClusterConfiguration.Builder() 46 | .setConfiguration(config) 47 | .setNumTaskManagers(1) 48 | .setNumSlotsPerTaskManager(1) 49 | .build(); 50 | 51 | flink = new MiniCluster(miniClusterConfiguration); 52 | 53 | flink.start(); 54 | } 55 | 56 | @AfterClass 57 | public static void tearDown() throws Exception { 58 | if (flink != null) { 59 | flink.close(); 60 | } 61 | } 62 | 63 | @Test 64 | public void testUpateWatermark() throws Exception { 65 | final Configuration clientConfiguration = new Configuration(); 66 | clientConfiguration.setInteger(RestOptions.RETRY_MAX_ATTEMPTS, 0); 67 | 68 | final StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( 69 | flink.getRestAddress().get().getHost(), 70 | flink.getRestAddress().get().getPort(), 71 | clientConfiguration); 72 | 73 | env.addSource(new TestSourceFunction(new JobManagerWatermarkTracker("fakeId"))) 74 | .addSink(new SinkFunction() {}); 75 | env.execute(); 76 | } 77 | 78 | private static class TestSourceFunction extends RichSourceFunction { 79 | 80 | private final JobManagerWatermarkTracker tracker; 81 | 82 | public TestSourceFunction(JobManagerWatermarkTracker tracker) { 83 | this.tracker = tracker; 84 | } 85 | 86 | @Override 87 | public void open(Configuration parameters) throws Exception { 88 | super.open(parameters); 89 | tracker.open(getRuntimeContext()); 90 | } 91 | 92 | @Override 93 | public void run(SourceContext ctx) { 94 | Assert.assertEquals(998, tracker.updateWatermark(998)); 95 | Assert.assertEquals(999, tracker.updateWatermark(999)); 96 | } 97 | 98 | @Override 99 | public void cancel() { 100 | } 101 | } 102 | 103 | } 104 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/model/SequenceNumber.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import java.io.Serializable; 25 | 26 | import static org.apache.flink.util.Preconditions.checkNotNull; 27 | 28 | /** 29 | * A serializable representation of a Kinesis record's sequence number. It has two fields: the main sequence number, 30 | * and also a subsequence number. If this {@link SequenceNumber} is referring to an aggregated Kinesis record, the 31 | * subsequence number will be a non-negative value representing the order of the sub-record within the aggregation. 32 | */ 33 | @Internal 34 | public class SequenceNumber implements Serializable { 35 | 36 | private static final long serialVersionUID = 876972197938972667L; 37 | 38 | private static final String DELIMITER = "-"; 39 | 40 | private final String sequenceNumber; 41 | private final long subSequenceNumber; 42 | 43 | private final int cachedHash; 44 | 45 | /** 46 | * Create a new instance for a non-aggregated Kinesis record without a subsequence number. 47 | * @param sequenceNumber the sequence number 48 | */ 49 | public SequenceNumber(String sequenceNumber) { 50 | this(sequenceNumber, -1); 51 | } 52 | 53 | /** 54 | * Create a new instance, with the specified sequence number and subsequence number. 55 | * To represent the sequence number for a non-aggregated Kinesis record, the subsequence number should be -1. 56 | * Otherwise, give a non-negative sequence number to represent an aggregated Kinesis record. 57 | * 58 | * @param sequenceNumber the sequence number 59 | * @param subSequenceNumber the subsequence number (-1 to represent non-aggregated Kinesis records) 60 | */ 61 | public SequenceNumber(String sequenceNumber, long subSequenceNumber) { 62 | this.sequenceNumber = checkNotNull(sequenceNumber); 63 | this.subSequenceNumber = subSequenceNumber; 64 | 65 | this.cachedHash = 37 * (sequenceNumber.hashCode() + Long.valueOf(subSequenceNumber).hashCode()); 66 | } 67 | 68 | public boolean isAggregated() { 69 | return subSequenceNumber >= 0; 70 | } 71 | 72 | public String getSequenceNumber() { 73 | return sequenceNumber; 74 | } 75 | 76 | public long getSubSequenceNumber() { 77 | return subSequenceNumber; 78 | } 79 | 80 | @Override 81 | public String toString() { 82 | if (isAggregated()) { 83 | return sequenceNumber + DELIMITER + subSequenceNumber; 84 | } else { 85 | return sequenceNumber; 86 | } 87 | } 88 | 89 | @Override 90 | public boolean equals(Object obj) { 91 | if (!(obj instanceof SequenceNumber)) { 92 | return false; 93 | } 94 | 95 | if (obj == this) { 96 | return true; 97 | } 98 | 99 | SequenceNumber other = (SequenceNumber) obj; 100 | 101 | return sequenceNumber.equals(other.getSequenceNumber()) 102 | && (subSequenceNumber == other.getSubSequenceNumber()); 103 | } 104 | 105 | @Override 106 | public int hashCode() { 107 | return cachedHash; 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/manualtests/ManualProducerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.manualtests; 21 | 22 | import org.apache.flink.api.java.utils.ParameterTool; 23 | import org.apache.flink.configuration.ConfigConstants; 24 | import org.apache.flink.streaming.api.datastream.DataStream; 25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 26 | 27 | import software.amazon.kinesis.connectors.flink.FlinkKinesisProducer; 28 | import software.amazon.kinesis.connectors.flink.KinesisPartitioner; 29 | import software.amazon.kinesis.connectors.flink.config.AWSConfigConstants; 30 | import software.amazon.kinesis.connectors.flink.examples.ProduceIntoKinesis; 31 | import software.amazon.kinesis.connectors.flink.serialization.KinesisSerializationSchema; 32 | 33 | import java.nio.ByteBuffer; 34 | import java.util.Properties; 35 | 36 | /** 37 | * This is a manual test for the AWS Kinesis connector in Flink. 38 | * 39 | *

It uses: 40 | * - A custom KinesisSerializationSchema 41 | * - A custom KinesisPartitioner 42 | * 43 | *

The streams "test-flink" and "flink-test-2" must exist. 44 | * 45 | *

Invocation: 46 | * --region eu-central-1 --accessKey X --secretKey X 47 | */ 48 | public class ManualProducerTest { 49 | 50 | public static void main(String[] args) throws Exception { 51 | ParameterTool pt = ParameterTool.fromArgs(args); 52 | 53 | StreamExecutionEnvironment see = StreamExecutionEnvironment.getExecutionEnvironment(); 54 | see.setParallelism(4); 55 | 56 | DataStream simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator()); 57 | 58 | Properties kinesisProducerConfig = new Properties(); 59 | kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region")); 60 | kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); 61 | kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); 62 | 63 | FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>( 64 | new KinesisSerializationSchema() { 65 | @Override 66 | public ByteBuffer serialize(String element) { 67 | return ByteBuffer.wrap(element.getBytes(ConfigConstants.DEFAULT_CHARSET)); 68 | } 69 | 70 | // every 10th element goes into a different stream 71 | @Override 72 | public String getTargetStream(String element) { 73 | if (element.split("-")[0].endsWith("0")) { 74 | return "flink-test-2"; 75 | } 76 | return null; // send to default stream 77 | } 78 | }, 79 | kinesisProducerConfig 80 | ); 81 | 82 | kinesis.setFailOnError(true); 83 | kinesis.setDefaultStream("test-flink"); 84 | kinesis.setDefaultPartition("0"); 85 | kinesis.setCustomPartitioner(new KinesisPartitioner() { 86 | @Override 87 | public String getPartitionId(String element) { 88 | int l = element.length(); 89 | return element.substring(l - 1, l); 90 | } 91 | }); 92 | simpleStringStream.addSink(kinesis); 93 | 94 | see.execute(); 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/model/StreamShardHandle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import com.amazonaws.services.kinesis.model.Shard; 25 | 26 | import static org.apache.flink.util.Preconditions.checkNotNull; 27 | 28 | /** 29 | * A wrapper class around the information provided along with streamName and 30 | * {@link com.amazonaws.services.kinesis.model.Shard}, with some extra utility methods to determine whether 31 | * or not a shard is closed and whether or not the shard is a result of parent shard splits or merges. 32 | */ 33 | @Internal 34 | public class StreamShardHandle { 35 | 36 | private final String streamName; 37 | private final Shard shard; 38 | 39 | private final int cachedHash; 40 | 41 | /** 42 | * Create a new StreamShardHandle. 43 | * 44 | * @param streamName 45 | * the name of the Kinesis stream that this shard belongs to 46 | * @param shard 47 | * the actual AWS Shard instance that will be wrapped within this StreamShardHandle 48 | */ 49 | public StreamShardHandle(String streamName, Shard shard) { 50 | this.streamName = checkNotNull(streamName); 51 | this.shard = checkNotNull(shard); 52 | 53 | // since our description of Kinesis Streams shards can be fully defined with the stream name and shard id, 54 | // our hash doesn't need to use hash code of Amazon's description of Shards, which uses other info for calculation 55 | int hash = 17; 56 | hash = 37 * hash + streamName.hashCode(); 57 | hash = 37 * hash + shard.getShardId().hashCode(); 58 | this.cachedHash = hash; 59 | } 60 | 61 | public String getStreamName() { 62 | return streamName; 63 | } 64 | 65 | public boolean isClosed() { 66 | return (shard.getSequenceNumberRange().getEndingSequenceNumber() != null); 67 | } 68 | 69 | public Shard getShard() { 70 | return shard; 71 | } 72 | 73 | @Override 74 | public String toString() { 75 | return "StreamShardHandle{" + 76 | "streamName='" + streamName + "'" + 77 | ", shard='" + shard.toString() + "'}"; 78 | } 79 | 80 | @Override 81 | public boolean equals(Object obj) { 82 | if (!(obj instanceof StreamShardHandle)) { 83 | return false; 84 | } 85 | 86 | if (obj == this) { 87 | return true; 88 | } 89 | 90 | StreamShardHandle other = (StreamShardHandle) obj; 91 | 92 | return streamName.equals(other.getStreamName()) && shard.equals(other.getShard()); 93 | } 94 | 95 | @Override 96 | public int hashCode() { 97 | return cachedHash; 98 | } 99 | 100 | /** 101 | * Utility function to compare two shard ids. 102 | * 103 | * @param firstShardId first shard id to compare 104 | * @param secondShardId second shard id to compare 105 | * @return a value less than 0 if the first shard id is smaller than the second shard id, 106 | * or a value larger than 0 the first shard is larger than the second shard id, 107 | * or 0 if they are equal 108 | */ 109 | public static int compareShardIds(String firstShardId, String secondShardId) { 110 | return firstShardId.compareTo(secondShardId); 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/util/WatermarkTrackerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.util; 21 | 22 | import org.apache.flink.streaming.util.MockStreamingRuntimeContext; 23 | 24 | import org.apache.commons.lang3.mutable.MutableLong; 25 | import org.junit.Assert; 26 | import org.junit.Test; 27 | 28 | import java.util.HashMap; 29 | import java.util.Map; 30 | 31 | /** Test for {@link WatermarkTracker}. */ 32 | public class WatermarkTrackerTest { 33 | 34 | WatermarkTracker.WatermarkState wm1 = new WatermarkTracker.WatermarkState(); 35 | MutableLong clock = new MutableLong(0); 36 | 37 | private class TestWatermarkTracker extends WatermarkTracker { 38 | /** 39 | * The watermarks of all sub tasks that participate in the synchronization. 40 | */ 41 | private final Map watermarks = new HashMap<>(); 42 | 43 | private long updateTimeoutCount = 0; 44 | 45 | @Override 46 | protected long getCurrentTime() { 47 | return clock.longValue(); 48 | } 49 | 50 | @Override 51 | public long updateWatermark(final long localWatermark) { 52 | refreshWatermarkSnapshot(this.watermarks); 53 | 54 | long currentTime = getCurrentTime(); 55 | String subtaskId = this.getSubtaskId(); 56 | 57 | WatermarkState ws = watermarks.get(subtaskId); 58 | if (ws == null) { 59 | watermarks.put(subtaskId, ws = new WatermarkState()); 60 | } 61 | ws.lastUpdated = currentTime; 62 | ws.watermark = Math.max(ws.watermark, localWatermark); 63 | saveWatermark(subtaskId, ws); 64 | 65 | long globalWatermark = ws.watermark; 66 | for (Map.Entry e : watermarks.entrySet()) { 67 | ws = e.getValue(); 68 | if (ws.lastUpdated + getUpdateTimeoutMillis() < currentTime) { 69 | // ignore outdated subtask 70 | updateTimeoutCount++; 71 | continue; 72 | } 73 | globalWatermark = Math.min(ws.watermark, globalWatermark); 74 | } 75 | return globalWatermark; 76 | } 77 | 78 | protected void refreshWatermarkSnapshot(Map watermarks) { 79 | watermarks.put("wm1", wm1); 80 | } 81 | 82 | protected void saveWatermark(String id, WatermarkState ws) { 83 | // do nothing 84 | } 85 | 86 | public long getUpdateTimeoutCount() { 87 | return updateTimeoutCount; 88 | } 89 | } 90 | 91 | @Test 92 | public void test() { 93 | long watermark = 0; 94 | TestWatermarkTracker ws = new TestWatermarkTracker(); 95 | ws.open(new MockStreamingRuntimeContext(false, 1, 0)); 96 | Assert.assertEquals(Long.MIN_VALUE, ws.updateWatermark(Long.MIN_VALUE)); 97 | Assert.assertEquals(Long.MIN_VALUE, ws.updateWatermark(watermark)); 98 | // timeout wm1 99 | clock.add(WatermarkTracker.DEFAULT_UPDATE_TIMEOUT_MILLIS + 1); 100 | Assert.assertEquals(watermark, ws.updateWatermark(watermark)); 101 | Assert.assertEquals(watermark, ws.updateWatermark(watermark - 1)); 102 | 103 | // min watermark 104 | wm1.watermark = watermark + 1; 105 | wm1.lastUpdated = clock.longValue(); 106 | Assert.assertEquals(watermark, ws.updateWatermark(watermark)); 107 | Assert.assertEquals(watermark + 1, ws.updateWatermark(watermark + 1)); 108 | } 109 | 110 | } 111 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/publisher/polling/PollingRecordPublisherFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher.polling; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.metrics.MetricGroup; 24 | import org.apache.flink.util.Preconditions; 25 | 26 | import software.amazon.kinesis.connectors.flink.internals.KinesisDataFetcher.FlinkKinesisProxyFactory; 27 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisher; 28 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisherFactory; 29 | import software.amazon.kinesis.connectors.flink.metrics.PollingRecordPublisherMetricsReporter; 30 | import software.amazon.kinesis.connectors.flink.model.StartingPosition; 31 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 32 | import software.amazon.kinesis.connectors.flink.proxy.KinesisProxyInterface; 33 | 34 | import java.util.Properties; 35 | 36 | /** 37 | * A {@link RecordPublisher} factory used to create instances of {@link PollingRecordPublisher}. 38 | */ 39 | @Internal 40 | public class PollingRecordPublisherFactory implements RecordPublisherFactory { 41 | 42 | private final FlinkKinesisProxyFactory kinesisProxyFactory; 43 | 44 | public PollingRecordPublisherFactory(final FlinkKinesisProxyFactory kinesisProxyFactory) { 45 | this.kinesisProxyFactory = kinesisProxyFactory; 46 | } 47 | 48 | /** 49 | * Create a {@link PollingRecordPublisher}. 50 | * An {@link AdaptivePollingRecordPublisher} will be created should adaptive reads be enabled in the configuration. 51 | * 52 | * @param startingPosition the position in the shard to start consuming records from 53 | * @param consumerConfig the consumer configuration properties 54 | * @param metricGroup the metric group to report metrics to 55 | * @param streamShardHandle the shard this consumer is subscribed to 56 | * @return a {@link PollingRecordPublisher} 57 | */ 58 | @Override 59 | public PollingRecordPublisher create( 60 | final StartingPosition startingPosition, 61 | final Properties consumerConfig, 62 | final MetricGroup metricGroup, 63 | final StreamShardHandle streamShardHandle) throws InterruptedException { 64 | Preconditions.checkNotNull(startingPosition); 65 | Preconditions.checkNotNull(consumerConfig); 66 | Preconditions.checkNotNull(metricGroup); 67 | Preconditions.checkNotNull(streamShardHandle); 68 | 69 | final PollingRecordPublisherConfiguration configuration = new PollingRecordPublisherConfiguration(consumerConfig); 70 | final PollingRecordPublisherMetricsReporter metricsReporter = new PollingRecordPublisherMetricsReporter(metricGroup); 71 | final KinesisProxyInterface kinesisProxy = kinesisProxyFactory.create(consumerConfig); 72 | 73 | if (configuration.isAdaptiveReads()) { 74 | return new AdaptivePollingRecordPublisher( 75 | startingPosition, 76 | streamShardHandle, 77 | metricsReporter, 78 | kinesisProxy, 79 | configuration.getMaxNumberOfRecordsPerFetch(), 80 | configuration.getFetchIntervalMillis()); 81 | } else { 82 | return new PollingRecordPublisher( 83 | startingPosition, 84 | streamShardHandle, 85 | metricsReporter, 86 | kinesisProxy, 87 | configuration.getMaxNumberOfRecordsPerFetch(), 88 | configuration.getFetchIntervalMillis()); 89 | } 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/proxy/KinesisProxyInterface.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.proxy; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | 24 | import com.amazonaws.services.kinesis.model.GetRecordsResult; 25 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 26 | 27 | import java.util.Map; 28 | 29 | /** 30 | * Interface for a Kinesis proxy that operates on multiple Kinesis streams within the same AWS service region. 31 | */ 32 | @Internal 33 | public interface KinesisProxyInterface { 34 | 35 | /** 36 | * Get a shard iterator from the specified position in a shard. 37 | * The retrieved shard iterator can be used in {@link KinesisProxyInterface#getRecords(String, int)}} 38 | * to read data from the Kinesis shard. 39 | * 40 | * @param shard the shard to get the iterator 41 | * @param shardIteratorType the iterator type, defining how the shard is to be iterated 42 | * (one of: TRIM_HORIZON, LATEST, AT_TIMESTAMP, AT_SEQUENCE_NUMBER, AFTER_SEQUENCE_NUMBER) 43 | * @param startingMarker should be {@code null} if shardIteratorType is TRIM_HORIZON or LATEST, 44 | * should be a {@code Date} value if shardIteratorType is AT_TIMESTAMP, 45 | * should be a {@code String} representing the sequence number if shardIteratorType is AT_SEQUENCE_NUMBER, AFTER_SEQUENCE_NUMBER 46 | * @return shard iterator which can be used to read data from Kinesis 47 | * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the 48 | * operation has exceeded the rate limit; this exception will be thrown 49 | * if the backoff is interrupted. 50 | */ 51 | String getShardIterator(StreamShardHandle shard, String shardIteratorType, Object startingMarker) throws InterruptedException; 52 | 53 | /** 54 | * Get the next batch of data records using a specific shard iterator. 55 | * 56 | * @param shardIterator a shard iterator that encodes info about which shard to read and where to start reading 57 | * @param maxRecordsToGet the maximum amount of records to retrieve for this batch 58 | * @return the batch of retrieved records, also with a shard iterator that can be used to get the next batch 59 | * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the 60 | * operation has exceeded the rate limit; this exception will be thrown 61 | * if the backoff is interrupted. 62 | */ 63 | GetRecordsResult getRecords(String shardIterator, int maxRecordsToGet) throws InterruptedException; 64 | 65 | /** 66 | * Get shard list of multiple Kinesis streams, ignoring the 67 | * shards of each stream before a specified last seen shard id. 68 | * 69 | * @param streamNamesWithLastSeenShardIds a map with stream as key, and last seen shard id as value 70 | * @return result of the shard list query 71 | * @throws InterruptedException this method will retry with backoff if AWS Kinesis complains that the 72 | * operation has exceeded the rate limit; this exception will be thrown 73 | * if the backoff is interrupted. 74 | */ 75 | GetShardListResult getShardList(Map streamNamesWithLastSeenShardIds) throws InterruptedException; 76 | 77 | } 78 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/util/WatermarkTracker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.util; 21 | 22 | import org.apache.flink.annotation.PublicEvolving; 23 | import org.apache.flink.api.common.functions.RuntimeContext; 24 | import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; 25 | 26 | import java.io.Closeable; 27 | import java.io.Serializable; 28 | 29 | /** 30 | * The watermark tracker is responsible for aggregating watermarks across distributed operators. 31 | *

It can be used for sub tasks of a single Flink source as well as multiple heterogeneous 32 | * sources or other operators. 33 | *

The class essentially functions like a distributed hash table that enclosing operators can 34 | * use to adopt their processing / IO rates. 35 | */ 36 | @PublicEvolving 37 | public abstract class WatermarkTracker implements Closeable, Serializable { 38 | 39 | public static final long DEFAULT_UPDATE_TIMEOUT_MILLIS = 60_000; 40 | 41 | /** 42 | * Subtasks that have not provided a watermark update within the configured interval will be 43 | * considered idle and excluded from target watermark calculation. 44 | */ 45 | private long updateTimeoutMillis = DEFAULT_UPDATE_TIMEOUT_MILLIS; 46 | 47 | /** 48 | * Unique id for the subtask. 49 | * Using string (instead of subtask index) so synchronization can spawn across multiple sources. 50 | */ 51 | private String subtaskId; 52 | 53 | /** Watermark state. */ 54 | protected static class WatermarkState { 55 | protected long watermark = Long.MIN_VALUE; 56 | protected long lastUpdated; 57 | 58 | public long getWatermark() { 59 | return watermark; 60 | } 61 | 62 | @Override 63 | public String toString() { 64 | return "WatermarkState{watermark=" + watermark + ", lastUpdated=" + lastUpdated + '}'; 65 | } 66 | } 67 | 68 | protected String getSubtaskId() { 69 | return this.subtaskId; 70 | } 71 | 72 | protected long getUpdateTimeoutMillis() { 73 | return this.updateTimeoutMillis; 74 | } 75 | 76 | public abstract long getUpdateTimeoutCount(); 77 | 78 | /** 79 | * Subtasks that have not provided a watermark update within the configured interval will be 80 | * considered idle and excluded from target watermark calculation. 81 | * 82 | * @param updateTimeoutMillis 83 | */ 84 | public void setUpdateTimeoutMillis(long updateTimeoutMillis) { 85 | this.updateTimeoutMillis = updateTimeoutMillis; 86 | } 87 | 88 | /** 89 | * Set the current watermark of the owning subtask and return the global low watermark based on 90 | * the current state snapshot. Periodically called by the enclosing consumer instance, which is 91 | * responsible for any timer management etc. 92 | * 93 | * @param localWatermark 94 | * @return 95 | */ 96 | public abstract long updateWatermark(final long localWatermark); 97 | 98 | protected long getCurrentTime() { 99 | return System.currentTimeMillis(); 100 | } 101 | 102 | public void open(RuntimeContext context) { 103 | if (context instanceof StreamingRuntimeContext) { 104 | this.subtaskId = ((StreamingRuntimeContext) context).getOperatorUniqueID() 105 | + "-" + context.getIndexOfThisSubtask(); 106 | } else { 107 | this.subtaskId = context.getTaskNameWithSubtasks(); 108 | } 109 | } 110 | 111 | @Override 112 | public void close() { 113 | // no work to do here 114 | } 115 | 116 | } 117 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/publisher/fanout/FanOutRecordPublisherFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals.publisher.fanout; 21 | 22 | import org.apache.flink.annotation.Internal; 23 | import org.apache.flink.metrics.MetricGroup; 24 | import org.apache.flink.util.Preconditions; 25 | 26 | import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; 27 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisher; 28 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisherFactory; 29 | import software.amazon.kinesis.connectors.flink.model.StartingPosition; 30 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 31 | import software.amazon.kinesis.connectors.flink.proxy.FullJitterBackoff; 32 | import software.amazon.kinesis.connectors.flink.proxy.KinesisProxyV2; 33 | import software.amazon.kinesis.connectors.flink.proxy.KinesisProxyV2Interface; 34 | 35 | import java.util.Optional; 36 | import java.util.Properties; 37 | 38 | import static java.util.Collections.singletonList; 39 | 40 | /** 41 | * A {@link RecordPublisher} factory used to create instances of {@link FanOutRecordPublisher}. 42 | */ 43 | @Internal 44 | public class FanOutRecordPublisherFactory implements RecordPublisherFactory { 45 | 46 | private static final FullJitterBackoff BACKOFF = new FullJitterBackoff(); 47 | 48 | /** 49 | * A singleton {@link KinesisProxyV2} is used per Flink task. 50 | * The {@link KinesisAsyncClient} uses an internal thread pool; using a single client reduces overhead. 51 | */ 52 | private final KinesisProxyV2Interface kinesisProxy; 53 | 54 | /** 55 | * Instantiate a factory responsible for creating {@link FanOutRecordPublisher}. 56 | * 57 | * @param kinesisProxy the singleton proxy used by all record publishers created by this factory 58 | */ 59 | public FanOutRecordPublisherFactory(final KinesisProxyV2Interface kinesisProxy) { 60 | this.kinesisProxy = kinesisProxy; 61 | } 62 | 63 | /** 64 | * Create a {@link FanOutRecordPublisher}. 65 | * 66 | * @param startingPosition the starting position in the shard to start consuming from 67 | * @param consumerConfig the consumer configuration properties 68 | * @param metricGroup the metric group to report metrics to 69 | * @param streamShardHandle the shard this consumer is subscribed to 70 | * @return a {@link FanOutRecordPublisher} 71 | */ 72 | @Override 73 | public FanOutRecordPublisher create( 74 | final StartingPosition startingPosition, 75 | final Properties consumerConfig, 76 | final MetricGroup metricGroup, 77 | final StreamShardHandle streamShardHandle) { 78 | Preconditions.checkNotNull(startingPosition); 79 | Preconditions.checkNotNull(consumerConfig); 80 | Preconditions.checkNotNull(metricGroup); 81 | Preconditions.checkNotNull(streamShardHandle); 82 | 83 | String stream = streamShardHandle.getStreamName(); 84 | FanOutRecordPublisherConfiguration configuration = new FanOutRecordPublisherConfiguration(consumerConfig, singletonList(stream)); 85 | 86 | Optional streamConsumerArn = configuration.getStreamConsumerArn(stream); 87 | Preconditions.checkState(streamConsumerArn.isPresent()); 88 | 89 | return new FanOutRecordPublisher(startingPosition, streamConsumerArn.get(), streamShardHandle, kinesisProxy, configuration, BACKOFF); 90 | } 91 | 92 | @Override 93 | public void close() { 94 | kinesisProxy.close(); 95 | } 96 | 97 | } 98 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/model/StartingPositionTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import com.amazonaws.services.kinesis.model.ShardIteratorType; 23 | import org.junit.Rule; 24 | import org.junit.Test; 25 | import org.junit.rules.ExpectedException; 26 | 27 | import java.util.Date; 28 | 29 | import static org.junit.Assert.assertEquals; 30 | import static org.junit.Assert.assertNull; 31 | 32 | /** 33 | * Tests for {@link StartingPosition}. 34 | */ 35 | public class StartingPositionTest { 36 | 37 | @Rule 38 | public final ExpectedException thrown = ExpectedException.none(); 39 | 40 | @Test 41 | public void testStartingPositionFromTimestamp() { 42 | Date date = new Date(); 43 | StartingPosition position = StartingPosition.fromTimestamp(date); 44 | assertEquals(ShardIteratorType.AT_TIMESTAMP, position.getShardIteratorType()); 45 | assertEquals(date, position.getStartingMarker()); 46 | } 47 | 48 | @Test 49 | public void testStartingPositionRestartFromSequenceNumber() { 50 | SequenceNumber sequenceNumber = new SequenceNumber("100"); 51 | StartingPosition position = StartingPosition.restartFromSequenceNumber(sequenceNumber); 52 | assertEquals(ShardIteratorType.AFTER_SEQUENCE_NUMBER, position.getShardIteratorType()); 53 | assertEquals("100", position.getStartingMarker()); 54 | } 55 | 56 | @Test 57 | public void testStartingPositionRestartFromAggregatedSequenceNumber() { 58 | SequenceNumber sequenceNumber = new SequenceNumber("200", 3); 59 | StartingPosition position = StartingPosition.restartFromSequenceNumber(sequenceNumber); 60 | assertEquals(ShardIteratorType.AT_SEQUENCE_NUMBER, position.getShardIteratorType()); 61 | assertEquals("200", position.getStartingMarker()); 62 | } 63 | 64 | @Test 65 | public void testStartingPositionContinueFromAggregatedSequenceNumber() { 66 | SequenceNumber sequenceNumber = new SequenceNumber("200", 3); 67 | StartingPosition position = StartingPosition.continueFromSequenceNumber(sequenceNumber); 68 | assertEquals(ShardIteratorType.AFTER_SEQUENCE_NUMBER, position.getShardIteratorType()); 69 | assertEquals("200", position.getStartingMarker()); 70 | } 71 | 72 | @Test 73 | public void testStartingPositionRestartFromSentinelEarliest() { 74 | SequenceNumber sequenceNumber = SentinelSequenceNumber.SENTINEL_EARLIEST_SEQUENCE_NUM.get(); 75 | StartingPosition position = StartingPosition.restartFromSequenceNumber(sequenceNumber); 76 | assertEquals(ShardIteratorType.TRIM_HORIZON, position.getShardIteratorType()); 77 | assertNull(position.getStartingMarker()); 78 | } 79 | 80 | @Test 81 | public void testStartingPositionRestartFromSentinelLatest() { 82 | SequenceNumber sequenceNumber = SentinelSequenceNumber.SENTINEL_LATEST_SEQUENCE_NUM.get(); 83 | StartingPosition position = StartingPosition.restartFromSequenceNumber(sequenceNumber); 84 | assertEquals(ShardIteratorType.LATEST, position.getShardIteratorType()); 85 | assertNull(position.getStartingMarker()); 86 | } 87 | 88 | @Test 89 | public void testStartingPositionRestartFromSentinelEnding() { 90 | thrown.expect(IllegalArgumentException.class); 91 | 92 | SequenceNumber sequenceNumber = SentinelSequenceNumber.SENTINEL_SHARD_ENDING_SEQUENCE_NUM.get(); 93 | StartingPosition position = StartingPosition.restartFromSequenceNumber(sequenceNumber); 94 | assertEquals(ShardIteratorType.LATEST, position.getShardIteratorType()); 95 | assertNull(position.getStartingMarker()); 96 | } 97 | 98 | } 99 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/TestableKinesisDataFetcherForShardConsumerException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | import org.apache.flink.core.testutils.OneShotLatch; 23 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 24 | 25 | import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; 26 | 27 | import software.amazon.kinesis.connectors.flink.internals.KinesisDataFetcher; 28 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisherFactory; 29 | import software.amazon.kinesis.connectors.flink.model.KinesisStreamShardState; 30 | import software.amazon.kinesis.connectors.flink.proxy.KinesisProxyInterface; 31 | import software.amazon.kinesis.connectors.flink.serialization.KinesisDeserializationSchema; 32 | 33 | import java.util.HashMap; 34 | import java.util.LinkedList; 35 | import java.util.List; 36 | import java.util.Properties; 37 | import java.util.concurrent.ExecutorService; 38 | import java.util.concurrent.Executors; 39 | import java.util.concurrent.ThreadFactory; 40 | import java.util.concurrent.TimeUnit; 41 | import java.util.concurrent.TimeoutException; 42 | import java.util.concurrent.atomic.AtomicReference; 43 | 44 | /** 45 | * Extension of the {@link KinesisDataFetcher} for testing what happens when the thread is interrupted during 46 | * {@link #awaitTermination()}. 47 | */ 48 | public class TestableKinesisDataFetcherForShardConsumerException extends TestableKinesisDataFetcher { 49 | public volatile boolean wasInterrupted = false; 50 | 51 | private OneShotLatch awaitTerminationWaiter = new OneShotLatch(); 52 | 53 | public TestableKinesisDataFetcherForShardConsumerException(final List fakeStreams, 54 | final SourceFunction.SourceContext sourceContext, 55 | final Properties fakeConfiguration, 56 | final KinesisDeserializationSchema deserializationSchema, 57 | final int fakeTotalCountOfSubtasks, 58 | final int fakeIndexOfThisSubtask, 59 | final AtomicReference thrownErrorUnderTest, 60 | final LinkedList subscribedShardsStateUnderTest, 61 | final HashMap subscribedStreamsToLastDiscoveredShardIdsStateUnderTest, 62 | final KinesisProxyInterface fakeKinesis, 63 | final RecordPublisherFactory recordPublisherFactory) { 64 | super(fakeStreams, sourceContext, fakeConfiguration, deserializationSchema, fakeTotalCountOfSubtasks, 65 | fakeIndexOfThisSubtask, thrownErrorUnderTest, subscribedShardsStateUnderTest, 66 | subscribedStreamsToLastDiscoveredShardIdsStateUnderTest, fakeKinesis); 67 | } 68 | 69 | /** 70 | * Block until awaitTermination() has been called on this class. 71 | */ 72 | public void waitUntilAwaitTermination(long timeout, TimeUnit timeUnit) throws InterruptedException, TimeoutException { 73 | awaitTerminationWaiter.await(timeout, timeUnit); 74 | } 75 | 76 | @Override 77 | protected ExecutorService createShardConsumersThreadPool(final String subtaskName) { 78 | final ThreadFactory threadFactory = 79 | new ThreadFactoryBuilder().setNameFormat("KinesisShardConsumers-%d").build(); 80 | return Executors.newSingleThreadExecutor(threadFactory); 81 | } 82 | 83 | @Override 84 | public void awaitTermination() throws InterruptedException { 85 | awaitTerminationWaiter.trigger(); 86 | try { 87 | // Force this method to only exit by thread getting interrupted. 88 | while (true) { 89 | Thread.sleep(1000); 90 | } 91 | } catch (InterruptedException e) { 92 | wasInterrupted = true; 93 | throw e; 94 | } 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 22 | 25 | 26 | 4.0.0 27 | 28 | 29 | org.apache.flink 30 | flink-connectors 31 | 1.11.2 32 | 33 | 34 | 35 | software.amazon.kinesis 36 | amazon-kinesis-connector-flink-parent 37 | 2.5.0-SNAPSHOT 38 | pom 39 | Amazon Kinesis Connector for Apache Flink Parent 40 | 41 | 42 | 1.12.276 43 | 2.17.247 44 | 1.14.8 45 | 0.14.0 46 | 1.5.3 47 | 4.5.9 48 | 4.4.11 49 | 2.3.1 50 | 1.11.2 51 | 30.0-jre 52 | 2.13.4 53 | 2.13.4.2 54 | 2.17.1 55 | 56 | 57 | 58 | scm:git:https://github.com/awslabs/amazon-kinesis-connector-flink.git 59 | scm:git:git@github.com:awslabs/amazon-kinesis-connector-flink.git 60 | https://github.com/awslabs/amazon-kinesis-connector-flink/tree/master 61 | 62 | 63 | 64 | amazon-kinesis-connector-flink 65 | amazon-kinesis-sql-connector-flink 66 | 67 | 68 | 69 | 70 | 71 | com.amazonaws 72 | aws-java-sdk-bom 73 | ${aws.sdk.version} 74 | pom 75 | import 76 | 77 | 78 | software.amazon.awssdk 79 | bom 80 | ${aws.sdkv2.version} 81 | pom 82 | import 83 | 84 | 85 | 86 | org.apache.flink 87 | flink-test-utils-junit 88 | ${project.parent.version} 89 | test 90 | 91 | 92 | com.fasterxml.jackson.core 93 | jackson-core 94 | ${jackson2.version} 95 | 96 | 97 | com.fasterxml.jackson.core 98 | jackson-databind 99 | ${jackson-databind.version} 100 | 101 | 102 | com.fasterxml.jackson.core 103 | jackson-annotations 104 | ${jackson2.version} 105 | 106 | 107 | com.fasterxml.jackson.dataformat 108 | jackson-dataformat-cbor 109 | ${jackson2.version} 110 | 111 | 112 | 113 | 114 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/proxy/KinesisProxyV2FactoryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package software.amazon.kinesis.connectors.flink.proxy; 19 | 20 | import com.amazonaws.ClientConfiguration; 21 | import com.amazonaws.ClientConfigurationFactory; 22 | import org.junit.Test; 23 | import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; 24 | import software.amazon.awssdk.http.nio.netty.internal.NettyConfiguration; 25 | import software.amazon.kinesis.connectors.flink.config.AWSConfigConstants; 26 | import software.amazon.kinesis.connectors.flink.testutils.TestUtils; 27 | 28 | import java.lang.reflect.Field; 29 | import java.util.Properties; 30 | 31 | import static org.junit.Assert.assertEquals; 32 | import static org.junit.Assert.assertFalse; 33 | import static org.junit.Assert.assertTrue; 34 | import static software.amazon.kinesis.connectors.flink.util.AWSUtil.AWS_CLIENT_CONFIG_PREFIX; 35 | 36 | /** 37 | * Test for methods in the {@link KinesisProxyV2Factory} class. 38 | */ 39 | public class KinesisProxyV2FactoryTest { 40 | 41 | @Test 42 | public void testClientConfigurationPopulatedFromDefaults() throws Exception { 43 | Properties properties = properties(); 44 | 45 | KinesisProxyV2Interface proxy = KinesisProxyV2Factory.createKinesisProxyV2(properties); 46 | NettyConfiguration nettyConfiguration = getNettyConfiguration(proxy); 47 | 48 | assertEquals(defaultClientConfiguration().getConnectionTimeout(), nettyConfiguration.connectTimeoutMillis()); 49 | } 50 | 51 | @Test 52 | public void testClientConfigurationPopulatedFromProperties() throws Exception { 53 | Properties properties = properties(); 54 | properties.setProperty(AWS_CLIENT_CONFIG_PREFIX + "connectionTimeout", "12345"); 55 | 56 | KinesisProxyV2Interface proxy = KinesisProxyV2Factory.createKinesisProxyV2(properties); 57 | NettyConfiguration nettyConfiguration = getNettyConfiguration(proxy); 58 | 59 | assertEquals(12345, nettyConfiguration.connectTimeoutMillis()); 60 | } 61 | 62 | @Test 63 | public void testClientConfigurationPopulatedTcpKeepAliveDefaults() throws Exception { 64 | Properties properties = properties(); 65 | 66 | KinesisProxyV2Interface proxy = KinesisProxyV2Factory.createKinesisProxyV2(properties); 67 | NettyConfiguration nettyConfiguration = getNettyConfiguration(proxy); 68 | 69 | assertTrue(nettyConfiguration.tcpKeepAlive()); 70 | } 71 | 72 | @Test 73 | public void testClientConfigurationPopulatedTcpKeepAliveFromProperties() throws Exception { 74 | Properties properties = properties(); 75 | properties.setProperty(AWS_CLIENT_CONFIG_PREFIX + "useTcpKeepAlive", "false"); 76 | 77 | KinesisProxyV2Interface proxy = KinesisProxyV2Factory.createKinesisProxyV2(properties); 78 | NettyConfiguration nettyConfiguration = getNettyConfiguration(proxy); 79 | 80 | assertFalse(nettyConfiguration.tcpKeepAlive()); 81 | } 82 | 83 | private NettyConfiguration getNettyConfiguration(final KinesisProxyV2Interface kinesis) throws Exception { 84 | NettyNioAsyncHttpClient httpClient = getField("httpClient", kinesis); 85 | return getField("configuration", httpClient); 86 | } 87 | 88 | private T getField(String fieldName, Object obj) throws Exception { 89 | Field field = obj.getClass().getDeclaredField(fieldName); 90 | field.setAccessible(true); 91 | return (T) field.get(obj); 92 | } 93 | 94 | private ClientConfiguration defaultClientConfiguration() { 95 | return new ClientConfigurationFactory().getConfig(); 96 | } 97 | 98 | private Properties properties() { 99 | Properties properties = TestUtils.efoProperties(); 100 | properties.setProperty(AWSConfigConstants.AWS_REGION, "eu-west-2"); 101 | return properties; 102 | } 103 | 104 | } 105 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/util/StreamConsumerRegistrarUtilTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.util; 21 | 22 | import org.junit.Test; 23 | import software.amazon.kinesis.connectors.flink.internals.publisher.fanout.StreamConsumerRegistrar; 24 | 25 | import java.util.Arrays; 26 | import java.util.List; 27 | import java.util.Properties; 28 | 29 | import static org.junit.Assert.assertEquals; 30 | import static org.mockito.Mockito.mock; 31 | import static org.mockito.Mockito.verify; 32 | import static org.mockito.Mockito.verifyZeroInteractions; 33 | import static org.mockito.Mockito.when; 34 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.EFORegistrationType.EAGER; 35 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.EFO_CONSUMER_NAME; 36 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.EFO_REGISTRATION_TYPE; 37 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.RECORD_PUBLISHER_TYPE; 38 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.RecordPublisherType.EFO; 39 | import static software.amazon.kinesis.connectors.flink.config.ConsumerConfigConstants.efoConsumerArn; 40 | 41 | /** 42 | * Tests for {@link StreamConsumerRegistrar}. 43 | */ 44 | public class StreamConsumerRegistrarUtilTest { 45 | 46 | @Test 47 | public void testRegisterStreamConsumers() throws Exception { 48 | Properties configProps = new Properties(); 49 | configProps.setProperty(EFO_CONSUMER_NAME, "consumer-name"); 50 | 51 | StreamConsumerRegistrar registrar = mock(StreamConsumerRegistrar.class); 52 | when(registrar.registerStreamConsumer("stream-1", "consumer-name")) 53 | .thenReturn("stream-1-consumer-arn"); 54 | when(registrar.registerStreamConsumer("stream-2", "consumer-name")) 55 | .thenReturn("stream-2-consumer-arn"); 56 | 57 | StreamConsumerRegistrarUtil.registerStreamConsumers(registrar, configProps, Arrays.asList("stream-1", "stream-2")); 58 | 59 | assertEquals("stream-1-consumer-arn", configProps.getProperty(efoConsumerArn("stream-1"))); 60 | assertEquals("stream-2-consumer-arn", configProps.getProperty(efoConsumerArn("stream-2"))); 61 | } 62 | 63 | @Test 64 | public void testDeregisterStreamConsumersMissingStreamArn() throws Exception { 65 | Properties configProps = new Properties(); 66 | configProps.setProperty(RECORD_PUBLISHER_TYPE, EFO.name()); 67 | configProps.setProperty(EFO_CONSUMER_NAME, "consumer-name"); 68 | 69 | List streams = Arrays.asList("stream-1", "stream-2"); 70 | StreamConsumerRegistrar registrar = mock(StreamConsumerRegistrar.class); 71 | 72 | StreamConsumerRegistrarUtil.deregisterStreamConsumers(registrar, configProps, streams); 73 | 74 | verify(registrar).deregisterStreamConsumer("stream-1"); 75 | verify(registrar).deregisterStreamConsumer("stream-2"); 76 | } 77 | 78 | @Test 79 | public void testDeregisterStreamConsumersOnlyDeregistersEFOLazilyInitializedConsumers() { 80 | Properties configProps = getDefaultConfiguration(); 81 | configProps.setProperty(RECORD_PUBLISHER_TYPE, EFO.name()); 82 | configProps.put(EFO_REGISTRATION_TYPE, EAGER.name()); 83 | List streams = Arrays.asList("stream-1"); 84 | StreamConsumerRegistrar registrar = mock(StreamConsumerRegistrar.class); 85 | 86 | StreamConsumerRegistrarUtil.deregisterStreamConsumers(registrar, configProps, streams); 87 | 88 | verifyZeroInteractions(registrar); 89 | } 90 | 91 | private Properties getDefaultConfiguration() { 92 | Properties configProps = new Properties(); 93 | configProps.setProperty(EFO_CONSUMER_NAME, "consumer-name"); 94 | return configProps; 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/model/DynamoDBStreamsShardHandleTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.model; 21 | 22 | import org.junit.Test; 23 | 24 | import static org.junit.Assert.assertEquals; 25 | import static org.junit.Assert.assertTrue; 26 | import static org.junit.Assert.fail; 27 | import static software.amazon.kinesis.connectors.flink.model.DynamoDBStreamsShardHandle.SHARDID_PREFIX; 28 | 29 | /** 30 | * Shard handle unit tests. 31 | */ 32 | public class DynamoDBStreamsShardHandleTest { 33 | @Test 34 | public void testIsValidShardId() { 35 | // normal form 36 | String shardId = "shardId-00000001536805703746-69688cb1"; 37 | assertEquals(true, DynamoDBStreamsShardHandle.isValidShardId(shardId)); 38 | 39 | // short form 40 | shardId = "shardId-00000001536805703746"; 41 | assertEquals(true, DynamoDBStreamsShardHandle.isValidShardId(shardId)); 42 | 43 | // long form 44 | shardId = "shardId-00000001536805703746-69688cb1aljkwerijfl8228sl12a123akfla"; 45 | assertEquals(true, DynamoDBStreamsShardHandle.isValidShardId(shardId)); 46 | 47 | // invalid with wrong prefix 48 | shardId = "sId-00000001536805703746-69688cb1"; 49 | assertEquals(false, DynamoDBStreamsShardHandle.isValidShardId(shardId)); 50 | 51 | // invalid with non-digits 52 | shardId = "shardId-0000000153680570aabb-69688cb1"; 53 | assertEquals(false, DynamoDBStreamsShardHandle.isValidShardId(shardId)); 54 | 55 | // invalid with shardId too long 56 | shardId = "shardId-00000001536805703746-69688cb1aljkwerijfl8228sl12a123akfla0000"; 57 | assertEquals(false, DynamoDBStreamsShardHandle.isValidShardId(shardId)); 58 | } 59 | 60 | @Test 61 | public void testCompareShardId() { 62 | final int numShardIds = 10; 63 | final int shardIdDigitLen = 20; 64 | final String zeros = "00000000000000000000"; // twenty '0' chars 65 | String shardIdValid = "shardId-00000001536805703746-69688cb1"; 66 | String shardIdInvalid = "shardId-0000000153680570aabb-69688cb1"; 67 | 68 | assertEquals(0, DynamoDBStreamsShardHandle.compareShardIds(shardIdValid, shardIdValid)); 69 | 70 | // comparison of invalid shardIds should yield exception 71 | try { 72 | DynamoDBStreamsShardHandle.compareShardIds(shardIdValid, shardIdInvalid); 73 | fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); 74 | } catch (IllegalArgumentException e) { 75 | // expected 76 | } 77 | try { 78 | DynamoDBStreamsShardHandle.compareShardIds(shardIdInvalid, shardIdValid); 79 | fail("invalid shard Id" + shardIdInvalid + " should trigger exception"); 80 | } catch (IllegalArgumentException e) { 81 | // expected 82 | } 83 | 84 | // compare randomly generated shardIds based on timestamp 85 | String[] shardIds = new String[numShardIds]; 86 | for (int i = 0; i < numShardIds; i++) { 87 | String nowStr = String.valueOf(System.currentTimeMillis()); 88 | if (nowStr.length() < shardIdDigitLen) { 89 | shardIds[i] = SHARDID_PREFIX + zeros.substring(0, shardIdDigitLen - nowStr.length()) 90 | + nowStr; 91 | } else { 92 | shardIds[i] = SHARDID_PREFIX + nowStr.substring(0, shardIdDigitLen); 93 | } 94 | try { 95 | Thread.sleep(100); 96 | } catch (InterruptedException e) { 97 | // ignore 98 | } 99 | } 100 | for (int i = 1; i < numShardIds - 1; i++) { 101 | assertTrue(DynamoDBStreamsShardHandle.compareShardIds(shardIds[i - 1], shardIds[i]) < 0); 102 | assertTrue(DynamoDBStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i]) == 0); 103 | assertTrue(DynamoDBStreamsShardHandle.compareShardIds(shardIds[i], shardIds[i + 1]) < 0); 104 | } 105 | } 106 | 107 | } 108 | -------------------------------------------------------------------------------- /amazon-kinesis-sql-connector-flink/pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 22 | 25 | 26 | 4.0.0 27 | 28 | 29 | software.amazon.kinesis 30 | amazon-kinesis-connector-flink-parent 31 | 2.5.0-SNAPSHOT 32 | .. 33 | 34 | 35 | Amazon Kinesis SQL Connector for Apache Flink 36 | amazon-kinesis-sql-connector-flink 37 | jar 38 | 39 | 40 | 41 | software.amazon.kinesis 42 | amazon-kinesis-connector-flink 43 | ${project.version} 44 | 45 | 46 | 47 | 48 | 49 | 50 | org.apache.flink 51 | flink-test-utils-junit 52 | ${flink.version} 53 | test 54 | 55 | 56 | 57 | 58 | 59 | 60 | 61 | org.apache.maven.plugins 62 | maven-shade-plugin 63 | 64 | 65 | shade 66 | package 67 | 68 | shade 69 | 70 | 71 | false 72 | 73 | 74 | software.amazon.kinesis:amazon-kinesis-connector-flink 75 | com.fasterxml.jackson.core:jackson-core 76 | com.fasterxml.jackson.core:jackson-databind 77 | com.fasterxml.jackson.core:jackson-annotations 78 | com.fasterxml.jackson.dataformat:jackson-dataformat-cbor 79 | joda-time:joda-time 80 | commons-codec:commons-codec 81 | commons-io:commons-io 82 | commons-lang:commons-lang 83 | commons-logging:commons-logging 84 | org.apache.commons:commons-lang3 85 | com.google.guava:guava 86 | com.google.guava:failureaccess 87 | 88 | 89 | 90 | 91 | *:* 92 | 93 | codegen-resources/** 94 | mozilla/public-suffix-list.txt 95 | VersionInfo.java 96 | mime.types 97 | 98 | 99 | 100 | 101 | 102 | com.fasterxml.jackson 103 | software.amazon.kinesis.shaded.com.fasterxml.jackson 104 | 105 | 106 | org.apache.commons 107 | software.amazon.kinesis.shaded.org.apache.commons 108 | 109 | 110 | com.google 111 | software.amazon.kinesis.shaded.com.google 112 | 113 | 114 | org.joda.time 115 | software.amazon.kinesis.shaded.org.joda.time 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/test/java/software/amazon/kinesis/connectors/flink/testutils/KinesisEventsGeneratorProducerThread.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.testutils; 21 | 22 | import org.apache.flink.api.common.serialization.SimpleStringSchema; 23 | import org.apache.flink.configuration.Configuration; 24 | import org.apache.flink.streaming.api.datastream.DataStream; 25 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; 26 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 27 | 28 | import org.apache.commons.lang3.RandomStringUtils; 29 | import org.slf4j.Logger; 30 | import org.slf4j.LoggerFactory; 31 | import software.amazon.kinesis.connectors.flink.FlinkKinesisProducer; 32 | import software.amazon.kinesis.connectors.flink.config.AWSConfigConstants; 33 | 34 | import java.util.Properties; 35 | import java.util.concurrent.atomic.AtomicReference; 36 | 37 | /** 38 | * A thread that runs a topology with a manual data generator as source, and the FlinkKinesisProducer as sink. 39 | */ 40 | public class KinesisEventsGeneratorProducerThread { 41 | 42 | private static final Logger LOG = LoggerFactory.getLogger(KinesisEventsGeneratorProducerThread.class); 43 | 44 | public static Thread create(final int totalEventCount, 45 | final int parallelism, 46 | final String awsAccessKey, 47 | final String awsSecretKey, 48 | final String awsRegion, 49 | final String kinesisStreamName, 50 | final AtomicReference errorHandler, 51 | final int flinkPort, 52 | final Configuration flinkConfig) { 53 | Runnable kinesisEventsGeneratorProducer = new Runnable() { 54 | @Override 55 | public void run() { 56 | try { 57 | StreamExecutionEnvironment see = StreamExecutionEnvironment.createRemoteEnvironment("localhost", flinkPort, flinkConfig); 58 | see.setParallelism(parallelism); 59 | 60 | // start data generator 61 | DataStream simpleStringStream = see.addSource(new KinesisEventsGeneratorProducerThread.EventsGenerator(totalEventCount)).setParallelism(1); 62 | 63 | Properties producerProps = new Properties(); 64 | producerProps.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, awsAccessKey); 65 | producerProps.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, awsSecretKey); 66 | producerProps.setProperty(AWSConfigConstants.AWS_REGION, awsRegion); 67 | 68 | FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(new SimpleStringSchema(), 69 | producerProps); 70 | 71 | kinesis.setFailOnError(true); 72 | kinesis.setDefaultStream(kinesisStreamName); 73 | kinesis.setDefaultPartition("0"); 74 | simpleStringStream.addSink(kinesis); 75 | 76 | LOG.info("Starting producing topology"); 77 | see.execute("Producing topology"); 78 | LOG.info("Producing topo finished"); 79 | } catch (Exception e) { 80 | LOG.warn("Error while running producing topology", e); 81 | errorHandler.set(e); 82 | } 83 | } 84 | }; 85 | 86 | return new Thread(kinesisEventsGeneratorProducer); 87 | } 88 | 89 | private static class EventsGenerator implements SourceFunction { 90 | 91 | private static final Logger LOG = LoggerFactory.getLogger(EventsGenerator.class); 92 | 93 | private boolean running = true; 94 | private final long limit; 95 | 96 | public EventsGenerator(long limit) { 97 | this.limit = limit; 98 | } 99 | 100 | @Override 101 | public void run(SourceContext ctx) throws Exception { 102 | long seq = 0; 103 | while (running) { 104 | Thread.sleep(10); 105 | String evt = (seq++) + "-" + RandomStringUtils.randomAlphabetic(12); 106 | ctx.collect(evt); 107 | LOG.info("Emitting event {}", evt); 108 | if (seq >= limit) { 109 | break; 110 | } 111 | } 112 | ctx.close(); 113 | LOG.info("Stopping events generator"); 114 | } 115 | 116 | @Override 117 | public void cancel() { 118 | running = false; 119 | } 120 | } 121 | } 122 | -------------------------------------------------------------------------------- /amazon-kinesis-connector-flink/src/main/java/software/amazon/kinesis/connectors/flink/internals/DynamoDBStreamsDataFetcher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * This file has been modified from the original. 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with 6 | * this work for additional information regarding copyright ownership. 7 | * The ASF licenses this file to You under the Apache License, Version 2.0 8 | * (the "License"); you may not use this file except in compliance with 9 | * the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | 20 | package software.amazon.kinesis.connectors.flink.internals; 21 | 22 | import org.apache.flink.annotation.VisibleForTesting; 23 | import org.apache.flink.api.common.functions.RuntimeContext; 24 | import org.apache.flink.metrics.MetricGroup; 25 | import org.apache.flink.streaming.api.functions.source.SourceFunction; 26 | 27 | import software.amazon.kinesis.connectors.flink.KinesisShardAssigner; 28 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisher; 29 | import software.amazon.kinesis.connectors.flink.internals.publisher.RecordPublisherFactory; 30 | import software.amazon.kinesis.connectors.flink.internals.publisher.polling.PollingRecordPublisherFactory; 31 | import software.amazon.kinesis.connectors.flink.model.DynamoDBStreamsShardHandle; 32 | import software.amazon.kinesis.connectors.flink.model.SequenceNumber; 33 | import software.amazon.kinesis.connectors.flink.model.StartingPosition; 34 | import software.amazon.kinesis.connectors.flink.model.StreamShardHandle; 35 | import software.amazon.kinesis.connectors.flink.proxy.DynamoDBStreamsProxy; 36 | import software.amazon.kinesis.connectors.flink.serialization.KinesisDeserializationSchema; 37 | 38 | import java.util.ArrayList; 39 | import java.util.List; 40 | import java.util.Properties; 41 | import java.util.concurrent.atomic.AtomicReference; 42 | 43 | /** 44 | * Dynamodb streams data fetcher. 45 | * @param type of fetched data. 46 | */ 47 | public class DynamoDBStreamsDataFetcher extends KinesisDataFetcher { 48 | private final RecordPublisherFactory recordPublisherFactory; 49 | 50 | /** 51 | * Constructor. 52 | * 53 | * @param streams list of streams to fetch data 54 | * @param sourceContext source context 55 | * @param runtimeContext runtime context 56 | * @param configProps config properties 57 | * @param deserializationSchema deserialization schema 58 | * @param shardAssigner shard assigner 59 | */ 60 | public DynamoDBStreamsDataFetcher(List streams, 61 | SourceFunction.SourceContext sourceContext, 62 | RuntimeContext runtimeContext, 63 | Properties configProps, 64 | KinesisDeserializationSchema deserializationSchema, 65 | KinesisShardAssigner shardAssigner) { 66 | 67 | this(streams, sourceContext, runtimeContext, configProps, deserializationSchema, shardAssigner, DynamoDBStreamsProxy::create); 68 | } 69 | 70 | @VisibleForTesting 71 | DynamoDBStreamsDataFetcher(List streams, 72 | SourceFunction.SourceContext sourceContext, 73 | RuntimeContext runtimeContext, 74 | Properties configProps, 75 | KinesisDeserializationSchema deserializationSchema, 76 | KinesisShardAssigner shardAssigner, 77 | FlinkKinesisProxyFactory flinkKinesisProxyFactory) { 78 | super(streams, 79 | sourceContext, 80 | sourceContext.getCheckpointLock(), 81 | runtimeContext, 82 | configProps, 83 | deserializationSchema, 84 | shardAssigner, 85 | null, 86 | null, 87 | new AtomicReference<>(), 88 | new ArrayList<>(), 89 | createInitialSubscribedStreamsToLastDiscoveredShardsState(streams), 90 | flinkKinesisProxyFactory, 91 | null); 92 | 93 | this.recordPublisherFactory = new PollingRecordPublisherFactory(flinkKinesisProxyFactory); 94 | } 95 | 96 | @Override 97 | protected boolean shouldAdvanceLastDiscoveredShardId(String shardId, String lastSeenShardIdOfStream) { 98 | if (DynamoDBStreamsShardHandle.compareShardIds(shardId, lastSeenShardIdOfStream) <= 0) { 99 | // shardID update is valid only if the given shard id is greater 100 | // than the previous last seen shard id of the stream. 101 | return false; 102 | } 103 | 104 | return true; 105 | } 106 | 107 | @Override 108 | protected RecordPublisher createRecordPublisher( 109 | SequenceNumber sequenceNumber, 110 | Properties configProps, MetricGroup metricGroup, 111 | StreamShardHandle subscribedShard) throws InterruptedException { 112 | StartingPosition startingPosition = StartingPosition.continueFromSequenceNumber(sequenceNumber); 113 | return recordPublisherFactory.create(startingPosition, getConsumerConfiguration(), metricGroup, subscribedShard); 114 | } 115 | 116 | } 117 | --------------------------------------------------------------------------------