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



##########
File path: 
flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkWriter.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.unified;
+
+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.apache.flink.streaming.connectors.kinesis.unified.util.AwsV2Util;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ClientConfigurationFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
+import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+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 software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+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";

Review comment:
       Can this metric be subsumed by 
`SinkWriterMetricGroup#getNumRecordsOutErrorsCounter`?
   
   In general, can you also have a look at 
`SinkWriterMetricGroup#setCurrentSendTimeGauge`?
   With 1.14 we tried to build a unified metric system for sinks which should 
make it easier for users to understand and compare metrics across multiple 
sinks. You can have a look at the metrics the `KafkaWriter` implements I guess 
most of them should also apply here 
https://github.com/apache/flink/blob/85fbeff2e7c3d552967b7a43564bf70c6d83985d/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/sink/KafkaWriter.java#L143




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to