CrynetLogistics commented on a change in pull request #17345: URL: https://github.com/apache/flink/pull/17345#discussion_r741752717
########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkBuilder.java ########## @@ -0,0 +1,110 @@ +/* + * 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.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; + +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; + +import java.util.Properties; + +/** + * Builder to construct {@link KinesisDataStreamsSink}. + * + * <p>The following example shows the minimum setup to create a {@link KinesisDataStreamsSink} that + * writes String values to a Kinesis Data Streams stream named your_stream_here. + * + * <pre>{@code + * KinesisDataStreamsSink<String> kdsSink = + * KinesisDataStreamsSink.<String>builder() + * .setElementConverter(elementConverter) + * .setStreamName("your_stream_name") + * .build(); + * }</pre> + * + * <p>If the following parameters are not set in this builder, the following defaults will be used: + * + * <ul> + * <li>{@code maxBatchSize} will be 200 + * <li>{@code maxInFlightRequests} will be 16 + * <li>{@code maxBufferedRequests} will be 10000 + * <li>{@code flushOnBufferSizeInBytes} will be 64MB i.e. {@code 64 * 1024 * 1024} + * <li>{@code maxTimeInBufferMS} will be 5000ms + * <li>{@code failOnError} will be false + * </ul> + * + * @param <InputT> type of elements that should be persisted in the destination + */ +@PublicEvolving +public class KinesisDataStreamsSinkBuilder<InputT> + extends AsyncSinkBaseBuilder< + InputT, PutRecordsRequestEntry, KinesisDataStreamsSinkBuilder<InputT>> { + + 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 static final boolean DEFAULT_FAIL_ON_ERROR = false; + + private Boolean failOnError; + private String streamName; + private Properties kinesisClientProperties; + + KinesisDataStreamsSinkBuilder() {} + + /** + * Sets the name of the KDS stream that the sink will connect to. There is no default for this + * parameter, therefore, this must be provided at sink creation time otherwise the build will + * fail. + * + * @param streamName the name of the stream + * @return {@link KinesisDataStreamsSinkBuilder} itself + */ + public KinesisDataStreamsSinkBuilder<InputT> setStreamName(String streamName) { + this.streamName = streamName; + return this; + } + + public KinesisDataStreamsSinkBuilder<InputT> setFailOnError(boolean failOnError) { + this.failOnError = failOnError; + return this; + } + + public KinesisDataStreamsSinkBuilder<InputT> setKinesisClientProperties( + Properties kinesisClientProperties) { + this.kinesisClientProperties = kinesisClientProperties; + return this; + } + + public KinesisDataStreamsSink<InputT> build() { Review comment: 🤦 🤦 Of course. Thanks. ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/util/AWSConfigConstants.java ########## @@ -0,0 +1,167 @@ +/* + * 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.util; + +import org.apache.flink.annotation.PublicEvolving; + +import com.amazonaws.auth.AWSCredentialsProvider; + +/** Configuration keys for AWS service usage. */ +@PublicEvolving +public class AWSConfigConstants { + + /** + * Possible configuration values for the type of credential provider to use when accessing AWS + * Kinesis. Internally, a corresponding implementation of {@link AWSCredentialsProvider} will be + * used. + */ + public enum CredentialProvider { + + /** + * Look for the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY to create + * AWS credentials. + */ + ENV_VAR, + + /** + * Look for Java system properties aws.accessKeyId and aws.secretKey to create AWS + * credentials. + */ + SYS_PROP, + + /** Use a AWS credentials profile file to create the AWS credentials. */ + PROFILE, + + /** + * Simply create AWS credentials by supplying the AWS access key ID and AWS secret key in + * the configuration properties. + */ + BASIC, + + /** + * Create AWS credentials by assuming a role. The credentials for assuming the role must be + * supplied. * + */ + ASSUME_ROLE, + + /** + * Use AWS WebIdentityToken in order to assume a role. A token file and role details can be + * supplied as configuration or environment variables. * + */ + WEB_IDENTITY_TOKEN, + + /** + * A credentials provider chain will be used that searches for credentials in this order: + * ENV_VARS, SYS_PROPS, WEB_IDENTITY_TOKEN, PROFILE in the AWS instance metadata. * + */ + AUTO, + } + + /** The AWS region of the Kinesis streams to be pulled ("us-east-1" is used if not set). */ + public static final String AWS_REGION = "aws.region"; + + /** + * The credential provider type to use when AWS credentials are required (BASIC is used if not + * set). + */ + public static final String AWS_CREDENTIALS_PROVIDER = "aws.credentials.provider"; + + /** The AWS access key ID to use when setting credentials provider type to BASIC. */ + public static final String AWS_ACCESS_KEY_ID = accessKeyId(AWS_CREDENTIALS_PROVIDER); + + /** The AWS secret key to use when setting credentials provider type to BASIC. */ + public static final String AWS_SECRET_ACCESS_KEY = secretKey(AWS_CREDENTIALS_PROVIDER); + + /** Optional configuration for profile path if credential provider type is set to be PROFILE. */ + public static final String AWS_PROFILE_PATH = profilePath(AWS_CREDENTIALS_PROVIDER); + + /** Optional configuration for profile name if credential provider type is set to be PROFILE. */ + public static final String AWS_PROFILE_NAME = profileName(AWS_CREDENTIALS_PROVIDER); + + /** + * The role ARN to use when credential provider type is set to ASSUME_ROLE or + * WEB_IDENTITY_TOKEN. + */ + public static final String AWS_ROLE_ARN = roleArn(AWS_CREDENTIALS_PROVIDER); Review comment: I decided to move it. But it won't break people's code since if they depend on `f-c-kinesis`, then it will also depend on `f-c-aws`. ########## File path: flink-connectors/flink-connector-aws/pom.xml ########## @@ -0,0 +1,130 @@ +<?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</artifactId> + <name>Flink : Connectors : AWS</name> + <properties> + <aws.sdkv2.version>2.17.52</aws.sdkv2.version> + </properties> + + <packaging>jar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-streaming-java</artifactId> + <version>${project.version}</version> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-table-api-java-bridge</artifactId> Review comment: Thanks, my bad this is gone now. ########## File path: flink-connectors/flink-connector-aws/pom.xml ########## @@ -0,0 +1,130 @@ +<?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</artifactId> + <name>Flink : Connectors : AWS</name> + <properties> + <aws.sdkv2.version>2.17.52</aws.sdkv2.version> + </properties> + + <packaging>jar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-streaming-java</artifactId> + <version>${project.version}</version> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-table-api-java-bridge</artifactId> + <version>${project.version}</version> + <scope>provided</scope> + <optional>true</optional> + </dependency> + + <!-- Amazon AWS SDK v2.x dependencies --> + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>kinesis</artifactId> + <version>${aws.sdkv2.version}</version> + </dependency> + + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-core</artifactId> + <version>1.12.7</version> + </dependency> + + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>netty-nio-client</artifactId> + <version>${aws.sdkv2.version}</version> + </dependency> + + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>sts</artifactId> + <version>${aws.sdkv2.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</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-kinesis</artifactId> + <version>1.12.7</version> Review comment: There are some utility classes that use v1 but implementation is entirely in v2. -- 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]
