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/13 16:29:06 UTC

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

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



##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkConfig.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.connector.base.sink.AsyncSinkBaseConfig;
+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}. */
+@PublicEvolving
+public class KinesisDataStreamsSinkConfig<InputT>
+        extends AsyncSinkBaseConfig<InputT, PutRecordsRequestEntry> implements Serializable {
+
+    private static final int DEFAULT_MAX_BATCH_SIZE = 200;
+    private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 16;
+    private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 10000;
+    private static final long DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B = 64 * 1024 * 1024;
+    private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 5000;
+
+    private final String streamName;
+
+    public KinesisDataStreamsSinkConfig(
+            ElementConverter<InputT, PutRecordsRequestEntry> elementConverter,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long flushOnBufferSizeInBytes,
+            long maxTimeInBufferMS,
+            String streamName) {
+        super(
+                elementConverter,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                flushOnBufferSizeInBytes,
+                maxTimeInBufferMS);
+
+        Preconditions.checkNotNull(
+                streamName,
+                "The stream name must be set and "
+                        + "set to a non null value when initializing the KDS Sink.");
+        this.streamName = streamName;
+    }
+
+    /**
+     * Create a {@link KinesisDataStreamsSinkConfig.Builder} to allow the fluent and articulate
+     * construction of a new {@link KinesisDataStreamsSinkConfig}.
+     *
+     * @param <InputT> type of incoming records
+     * @return {@link KinesisDataStreamsSinkConfig.Builder}
+     */
+    public static <InputT> KinesisDataStreamsSinkConfig.Builder<InputT> builder() {
+        return new KinesisDataStreamsSinkConfig.Builder<>();
+    }
+
+    public String getStreamName() {
+        return streamName;
+    }
+
+    /** A builder for the encapsulating class, {@link KinesisDataStreamsSinkConfig}. */
+    public static class Builder<InputT> {
+
+        private ElementConverter<InputT, PutRecordsRequestEntry> elementConverter;
+        private int maxBatchSize = DEFAULT_MAX_BATCH_SIZE;
+        private int maxInFlightRequests = DEFAULT_MAX_IN_FLIGHT_REQUESTS;
+        private int maxBufferedRequests = DEFAULT_MAX_BUFFERED_REQUESTS;
+        private long flushOnBufferSizeInBytes = DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B;
+        private long maxTimeInBufferMS = DEFAULT_MAX_TIME_IN_BUFFER_MS;
+        private String streamName;
+
+        public Builder<InputT> setElementConverter(
+                ElementConverter<InputT, PutRecordsRequestEntry> elementConverter) {
+            this.elementConverter = elementConverter;
+            return this;
+        }
+
+        public Builder<InputT> setMaxBatchSize(int maxBatchSize) {
+            this.maxBatchSize = maxBatchSize;
+            return this;
+        }
+
+        public Builder<InputT> setMaxInFlightRequests(int maxInFlightRequests) {
+            this.maxInFlightRequests = maxInFlightRequests;
+            return this;
+        }
+
+        public Builder<InputT> setMaxBufferedRequests(int maxBufferedRequests) {
+            this.maxBufferedRequests = maxBufferedRequests;
+            return this;
+        }
+
+        public Builder<InputT> setFlushOnBufferSizeInBytes(long flushOnBufferSizeInBytes) {
+            this.flushOnBufferSizeInBytes = flushOnBufferSizeInBytes;
+            return this;
+        }
+
+        public Builder<InputT> setMaxTimeInBufferMS(long maxTimeInBufferMS) {
+            this.maxTimeInBufferMS = maxTimeInBufferMS;
+            return this;
+        }
+
+        public Builder<InputT> setStreamName(String streamName) {
+            Preconditions.checkArgument(!streamName.isEmpty());

Review comment:
       This validation is not enforced by the constructor, and since the constructor is `public` it can be bypasses. Suggest we move this to constructor, since it is already validating non-null

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkConfig.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.connector.base.sink.AsyncSinkBaseConfig;
+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}. */
+@PublicEvolving
+public class KinesisDataStreamsSinkConfig<InputT>
+        extends AsyncSinkBaseConfig<InputT, PutRecordsRequestEntry> implements Serializable {
+
+    private static final int DEFAULT_MAX_BATCH_SIZE = 200;
+    private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 16;
+    private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 10000;
+    private static final long DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B = 64 * 1024 * 1024;
+    private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 5000;
+
+    private final String streamName;
+
+    public KinesisDataStreamsSinkConfig(
+            ElementConverter<InputT, PutRecordsRequestEntry> elementConverter,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long flushOnBufferSizeInBytes,
+            long maxTimeInBufferMS,
+            String streamName) {
+        super(
+                elementConverter,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                flushOnBufferSizeInBytes,
+                maxTimeInBufferMS);
+
+        Preconditions.checkNotNull(
+                streamName,
+                "The stream name must be set and "
+                        + "set to a non null value when initializing the KDS Sink.");

Review comment:
       nit: This can be simplified, for example `The stream name must non null value when initializing the KDS Sink.`

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkWriter.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.Internal;
+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.groups.SinkWriterMetricGroup;
+
+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.
+ */
+@Internal
+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 transient Counter totalFullySuccessfulFlushesCounter;
+    private transient Counter totalPartiallySuccessfulFlushesCounter;
+    private transient Counter totalFullyFailedFlushesCounter;
+    private transient Counter numRecordsOutErrorsCounter;
+
+    private final String streamName;
+    private final SinkWriterMetricGroup metrics;
+    private static final KinesisAsyncClient client = KinesisAsyncClient.create();

Review comment:
       Why is this client static? This means it will be shared between all subtasks on the same TM. Is this desired? If so, please justify

##########
File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/async/KinesisDataStreamsSinkWriter.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.Internal;
+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.groups.SinkWriterMetricGroup;
+
+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.
+ */
+@Internal
+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 transient Counter totalFullySuccessfulFlushesCounter;
+    private transient Counter totalPartiallySuccessfulFlushesCounter;
+    private transient Counter totalFullyFailedFlushesCounter;
+    private transient Counter numRecordsOutErrorsCounter;
+
+    private final String streamName;
+    private final SinkWriterMetricGroup metrics;
+    private static final KinesisAsyncClient client = KinesisAsyncClient.create();

Review comment:
       We are using the standard builder here. Customers might want to set SDK specific configuration and we should consider exposing a mechanism to pass in additional config.




-- 
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