You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/05 15:32:07 UTC

[GitHub] [flink] fapaul commented on a change in pull request #17345: [FLINK-24227][connectors] FLIP-171: Added Kinesis Data Streams Sink i…

fapaul commented on a change in pull request #17345:
URL: https://github.com/apache/flink/pull/17345#discussion_r722344874



##########
File path: flink-connectors/flink-connector-aws/pom.xml
##########
@@ -0,0 +1,120 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-connectors</artifactId>
+		<version>1.15-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-aws_${scala.binary.version}</artifactId>
+	<name>Flink : Connectors : AWS</name>
+	<properties>
+		<aws.sdkv2.version>2.16.86</aws.sdkv2.version>
+	</properties>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<optional>true</optional>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>testcontainers</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Amazon AWS SDK v2.x dependencies -->
+		<dependency>

Review comment:
       NIt: Can you move this up outside of the test dependency block?

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkConfig.java
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.async;
+
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.util.Preconditions;
+
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+
+import java.io.Serializable;
+
+/** Configuration for {@link KinesisDataStreamsSink}. */
+public class KinesisDataStreamsSinkConfig<InputT> implements Serializable {

Review comment:
       All public classes should be annotated with either `@Internal` or `@Public/Evolving`. Does it need to be public?

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkWriter.java
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.async;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsResultEntry;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+/**
+ * Sink writer created by {@link KinesisDataStreamsSink} to write to Kinesis Data Streams. More
+ * details on the operation of this sink writer may be found in the doc for {@link
+ * KinesisDataStreamsSink}. More details on the internals of this sink writer may be found in {@link
+ * AsyncSinkWriter}.
+ *
+ * <p>The {@link KinesisAsyncClient} used here may be configured in the standard way for the AWS SDK
+ * 2.x. e.g. the provision of {@code AWS_REGION}, {@code AWS_ACCESS_KEY_ID} and {@code
+ * AWS_SECRET_ACCESS_KEY} through environment variables etc.
+ */
+@PublicEvolving
+public class KinesisDataStreamsSinkWriter<InputT>
+        extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> {
+
+    private static final String TOTAL_FULLY_SUCCESSFUL_FLUSHES_METRIC =
+            "totalFullySuccessfulFlushes";
+    private static final String TOTAL_PARTIALLY_SUCCESSFUL_FLUSHES_METRIC =
+            "totalPartiallySuccessfulFlushes";
+    private static final String TOTAL_FULLY_FAILED_FLUSHES_METRIC = "totalFullyFailedFlushes";
+    private static final String TOTAL_FAILED_ELEMENTS_METRIC = "totalFailedElements";
+    private transient Counter totalFullySuccessfulFlushesCounter;

Review comment:
       Nice to see some metrics implemented 👍  Can you also take a look at https://github.com/apache/flink/blob/e7d1bfbbb0bbf57007cbd0028fba3db065e249e9/flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/SinkWriterMetricGroup.java#L28 ? This exposes a standardized sinkwriter metrics. Maybe we can also make te, part of the AsyncSink base clasee.

##########
File path: flink-connectors/flink-connector-aws/src/test/java/org/apache/flink/streaming/connectors/kinesis/async/testutils/KinesaliteContainer.java
##########
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.async.testutils;
+
+import org.rnorth.ducttape.unreliables.Unreliables;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.wait.strategy.AbstractWaitStrategy;
+import org.testcontainers.utility.DockerImageName;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.http.SdkHttpConfigurationOption;
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.ListStreamsResponse;
+import software.amazon.awssdk.utils.AttributeMap;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A {@code org.testcontainers} based on Kinesalite.
+ *
+ * <p>Note that the more obvious localstack container with Kinesis took 1 minute to start vs 3
+ * seconds of Kinesalite.
+ */
+public class KinesaliteContainer extends GenericContainer<KinesaliteContainer> {

Review comment:
       I think we should reuse the existing `KinesaliteContainer` here and not start another implementation.

##########
File path: flink-connectors/flink-connector-aws/src/main/resources/log4j2.properties
##########
@@ -0,0 +1,25 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+rootLogger.level = INFO

Review comment:
       Please set this to `OFF` the CI will automatically set the logger and we do not want log anything by default

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkWriter.java
##########
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.async;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink.Sink;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsResultEntry;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+/**
+ * Sink writer created by {@link KinesisDataStreamsSink} to write to Kinesis Data Streams. More
+ * details on the operation of this sink writer may be found in the doc for {@link
+ * KinesisDataStreamsSink}. More details on the internals of this sink writer may be found in {@link
+ * AsyncSinkWriter}.
+ *
+ * <p>The {@link KinesisAsyncClient} used here may be configured in the standard way for the AWS SDK
+ * 2.x. e.g. the provision of {@code AWS_REGION}, {@code AWS_ACCESS_KEY_ID} and {@code
+ * AWS_SECRET_ACCESS_KEY} through environment variables etc.
+ */
+@PublicEvolving
+public class KinesisDataStreamsSinkWriter<InputT>

Review comment:
       Usually, we do not mark `SinkWriter` as `@PublicEvolving` is there a particular reason for it here?

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkConfig.java
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kinesis.async;
+
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.util.Preconditions;
+
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+
+import java.io.Serializable;
+
+/** Configuration for {@link KinesisDataStreamsSink}. */
+public class KinesisDataStreamsSinkConfig<InputT> implements Serializable {
+
+    private static final int DEFAULT_MAX_BATCH_SIZE = 100;
+    private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 8;
+    private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 1000;
+    private static final long DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B = 64 * 1024 * 1024;
+    private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 1000;
+
+    private final ElementConverter<InputT, PutRecordsRequestEntry> elementConverter;
+    private final int maxBatchSize;
+    private final int maxInFlightRequests;
+    private final int maxBufferedRequests;
+    private final long flushOnBufferSizeInBytes;
+    private final long maxTimeInBufferMS;

Review comment:
       I wonder whether we should introduce a common builder/configuration for the AsyncSink. I guess all sinks will eventually implement the same set of configurations.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org