dannycranmer commented on code in PR #39: URL: https://github.com/apache/flink-connector-aws/pull/39#discussion_r1071935816
########## flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyAsyncV2.java: ########## @@ -0,0 +1,78 @@ +/* + * 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.proxy; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.kinesis.internals.publisher.fanout.FanOutRecordPublisherConfiguration; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; + +import java.util.concurrent.CompletableFuture; + +/** + * Kinesis proxy implementation using AWS SDK v2.x - a utility class that is used as a proxy to make + * calls to AWS Kinesis for several EFO (Enhanced Fan Out) functions, such as de-/registering stream + * consumers, subscribing to a shard and receiving records from a shard. + */ +@Internal +public class KinesisProxyAsyncV2 implements KinesisProxyAsyncV2Interface { + + private static final Logger LOG = LoggerFactory.getLogger(KinesisProxyAsyncV2.class); Review Comment: This is unused ########## flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSAsyncSinkUtil.java: ########## @@ -161,4 +160,64 @@ S createAwsAsyncClient( .region(getRegion(configProps)) .build(); } + + /** + * @param configProps configuration properties + * @param httpClient the underlying HTTP client used to talk to AWS + * @param clientBuilder httpClientBuilder to build the underlying HTTP client + * @param awsUserAgentPrefixFormat user agent prefix for Flink + * @param awsClientUserAgentPrefix user agent prefix for kinesis client + * @return a new AWS Sync Client + */ + public static < + S extends SdkClient, + T extends + AwsSyncClientBuilder<? extends T, S> & AwsClientBuilder<? extends T, S>> + S createAwsSyncClient( Review Comment: It seems odd to put `createAwsSyncClient` in `AWSAsyncSinkUtil`. Should this be in `AWSGeneralUtil` instead? I assume the AsyncSink does not use the Sync client? ########## flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSAsyncSinkUtil.java: ########## @@ -161,4 +163,47 @@ S createAwsAsyncClient( .region(getRegion(configProps)) .build(); } + + /** + * @param configProps configuration properties Review Comment: It is still missing a comment. The first line of javadoc is the comment ########## flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/internals/publisher/fanout/StreamConsumerRegistrar.java: ########## @@ -77,8 +79,8 @@ public StreamConsumerRegistrar( * @throws InterruptedException */ public String registerStreamConsumer(final String stream, final String streamConsumerName) - throws ExecutionException, InterruptedException { - LOG.debug("Registering stream consumer - {}::{}", stream, streamConsumerName); + throws Exception { Review Comment: My concerns is the Exception is a wider catch bucket than before. However given `ExecutionException, InterruptedException` would have caught the majority and the parent wraps in a `FlinkKinesisStreamConsumerRegistrarException`, it looks ok ########## flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxyAsyncV2Interface.java: ########## @@ -0,0 +1,41 @@ +/* + * 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.proxy; + +import org.apache.flink.annotation.Internal; + +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardRequest; +import software.amazon.awssdk.services.kinesis.model.SubscribeToShardResponseHandler; + +import java.util.concurrent.CompletableFuture; + +/** + * Interface for a Kinesis proxy using AWS SDK v2.x operating on multiple Kinesis streams within the + * same AWS service region. + */ +@Internal +public interface KinesisProxyAsyncV2Interface { + + CompletableFuture<Void> subscribeToShard( + SubscribeToShardRequest request, SubscribeToShardResponseHandler responseHandler); + + /** Destroy any open resources used by the factory. */ + default void close() { Review Comment: Seems like a shortcut for testing. I would rather move it to `KinesisProxyAsyncV2InterfaceAdapter` to force developers to think about `close()` in src. We missed this before and it caused issues! -- 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]
