vahmed-hamdy commented on a change in pull request #18348:
URL: https://github.com/apache/flink/pull/18348#discussion_r784954636



##########
File path: 
flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/table/KinesisDataStreamsConnectorOptionsUtils.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.connector.kinesis.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+import org.apache.flink.connector.aws.table.util.AWSOptionUtils;
+import org.apache.flink.connector.base.table.options.ConfigurationValidator;
+import org.apache.flink.connector.base.table.options.TableOptionsUtils;
+import 
org.apache.flink.connector.base.table.sink.options.AsyncSinkConfigurationValidator;
+import 
org.apache.flink.connector.kinesis.sink.KinesisDataStreamsSinkElementConverter;
+import org.apache.flink.connector.kinesis.table.util.KinesisClientOptionsUtils;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+
+import static 
org.apache.flink.connector.base.table.AsyncSinkConnectorOptions.FLUSH_BUFFER_SIZE;
+import static 
org.apache.flink.connector.base.table.AsyncSinkConnectorOptions.FLUSH_BUFFER_TIMEOUT;
+import static 
org.apache.flink.connector.base.table.AsyncSinkConnectorOptions.MAX_BATCH_SIZE;
+import static 
org.apache.flink.connector.base.table.AsyncSinkConnectorOptions.MAX_BUFFERED_REQUESTS;
+import static 
org.apache.flink.connector.base.table.AsyncSinkConnectorOptions.MAX_IN_FLIGHT_REQUESTS;
+import static 
org.apache.flink.connector.kinesis.table.KinesisConnectorOptions.SINK_FAIL_ON_ERROR;
+import static 
org.apache.flink.connector.kinesis.table.KinesisConnectorOptions.SINK_PARTITIONER;
+import static 
org.apache.flink.connector.kinesis.table.KinesisConnectorOptions.SINK_PARTITIONER_FIELD_DELIMITER;
+import static 
org.apache.flink.connector.kinesis.table.KinesisConnectorOptions.STREAM;
+import static org.apache.flink.table.factories.FactoryUtil.FORMAT;
+
+/**
+ * Class for handling kinesis table options, including key mapping and 
validations and property
+ * extraction. Class uses options decorators {@link AWSOptionUtils}, {@link
+ * KinesisClientOptionsUtils} for handling each specified set of options.
+ */
+public class KinesisDataStreamsConnectorOptionsUtils extends 
AsyncSinkConfigurationValidator {
+    private static final Logger LOG =
+            
LoggerFactory.getLogger(KinesisDataStreamsConnectorOptionsUtils.class);
+
+    public static final String KINESIS_CLIENT_PROPERTIES_KEY = 
"sink.client.properties";
+
+    private final AWSOptionUtils awsOptionUtils;
+    private final KinesisClientOptionsUtils kinesisClientOptionsUtils;
+    private final KinesisProducerOptionsMapper kinesisProducerOptionsMapper;
+    private final Map<String, String> resolvedOptions;
+    private final ReadableConfig tableOptions;
+    private final 
KinesisDataStreamsSinkElementConverter.PartitionKeyGenerator<RowData> 
partitioner;
+
+    /**
+     * Prefixes of properties that are validated by downstream components and 
should not be
+     * validated by the Table API infrastructure.
+     */
+    private static final String[] NON_VALIDATED_PREFIXES =
+            new String[] {
+                AWSOptionUtils.AWS_PROPERTIES_PREFIX,
+                KinesisClientOptionsUtils.SINK_CLIENT_PREFIX,
+                KinesisProducerOptionsMapper.KINESIS_PRODUCER_PREFIX
+            };
+
+    protected static final Set<String> TABLE_LEVEL_OPTIONS =
+            new HashSet<>(
+                    Arrays.asList(
+                            STREAM.key(),
+                            FORMAT.key(),
+                            SINK_PARTITIONER.key(),
+                            SINK_FAIL_ON_ERROR.key(),
+                            SINK_PARTITIONER_FIELD_DELIMITER.key(),
+                            FLUSH_BUFFER_SIZE.key(),
+                            FLUSH_BUFFER_TIMEOUT.key(),
+                            MAX_BATCH_SIZE.key(),
+                            MAX_BUFFERED_REQUESTS.key(),
+                            MAX_IN_FLIGHT_REQUESTS.key()));
+
+    public KinesisDataStreamsConnectorOptionsUtils(
+            Map<String, String> options,
+            ReadableConfig tableOptions,
+            RowType physicalType,
+            List<String> partitionKeys,
+            ClassLoader classLoader) {
+        super(tableOptions);
+        this.resolvedOptions = filterTableOptions(options);
+        this.tableOptions = tableOptions;
+        this.awsOptionUtils = new AWSOptionUtils(resolvedOptions);
+        this.kinesisClientOptionsUtils = new 
KinesisClientOptionsUtils(resolvedOptions);
+        this.kinesisProducerOptionsMapper = new 
KinesisProducerOptionsMapper(resolvedOptions);
+        this.partitioner =
+                KinesisPartitionKeyGeneratorFactory.getKinesisPartitioner(
+                        tableOptions, physicalType, partitionKeys, 
classLoader);
+    }
+
+    public Properties getValidatedSinkConfigurations() {
+        Properties properties = super.getValidatedConfigurations();
+        properties.put(STREAM.key(), tableOptions.get(STREAM));
+        Properties awsProps = awsOptionUtils.getValidatedConfigurations();
+        Properties kinesisClientProps = 
kinesisClientOptionsUtils.getValidatedConfigurations();
+
+        for (Map.Entry<Object, Object> entry : awsProps.entrySet()) {
+            if (!properties.containsKey(entry.getKey())) {
+                kinesisClientProps.put(entry.getKey(), entry.getValue());
+            }
+        }
+        Properties producerFallbackProperties =
+                kinesisProducerOptionsMapper.getValidatedConfigurations();
+        for (Map.Entry<Object, Object> entry : 
producerFallbackProperties.entrySet()) {
+            if (!properties.containsKey(entry.getKey())) {
+                properties.put(entry.getKey(), entry.getValue());
+            }
+        }
+
+        properties.put(KINESIS_CLIENT_PROPERTIES_KEY, kinesisClientProps);
+        properties.put(SINK_PARTITIONER.key(), this.partitioner);
+
+        if (tableOptions.getOptional(SINK_FAIL_ON_ERROR).isPresent()) {
+            properties.put(
+                    SINK_FAIL_ON_ERROR.key(), 
tableOptions.getOptional(SINK_FAIL_ON_ERROR).get());
+        }
+        if (!awsProps.containsKey(AWSConfigConstants.AWS_REGION)) {
+            // per requirement in Amazon Kinesis DataStream
+            throw new IllegalArgumentException(
+                    String.format(
+                            "For FlinkKinesisSink AWS region ('%s') must be 
set in the config.",
+                            AWSConfigConstants.AWS_REGION));
+        }

Review comment:
       It is migrated from legacy connector validation, will recheck and update 
accordingly.




-- 
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-unsubscr...@flink.apache.org

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


Reply via email to