vahmed-hamdy commented on code in PR #114:
URL: 
https://github.com/apache/flink-connector-aws/pull/114#discussion_r1519868784


##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableFactory.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.redshift.executor.RedshiftS3Util;
+import org.apache.flink.connector.redshift.mode.SinkMode;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/** Dynamic Table Factory. */
+@PublicEvolving
+public class RedshiftDynamicTableFactory implements DynamicTableSinkFactory {
+    public static final String IDENTIFIER = "redshift";
+
+    public static final ConfigOption<String> HOSTNAME =
+            ConfigOptions.key("hostname")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDeprecatedKeys("AWS Redshift cluster hostname.");
+
+    public static final ConfigOption<Integer> PORT =
+            ConfigOptions.key("port")
+                    .intType()
+                    .defaultValue(5439)
+                    .withDeprecatedKeys("AWS Redshift port number.\nDefault 
value : 5439.");
+
+    public static final ConfigOption<String> USERNAME =
+            ConfigOptions.key("username")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift Cluster username.");
+
+    public static final ConfigOption<String> PASSWORD =
+            ConfigOptions.key("password")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster password.");
+
+    public static final ConfigOption<String> DATABASE_NAME =
+            ConfigOptions.key("sink.database-name")
+                    .stringType()
+                    .defaultValue("dev")
+                    .withDescription(
+                            "AWS Redshift cluster database name. Default value 
set to `dev`.");
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("sink.table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster sink table name.");
+
+    public static final ConfigOption<Integer> SINK_BATCH_SIZE =

Review Comment:
   Suggestion: have you considered using 
[AsyncDynamicTableSink](https://github.com/apache/flink/blob/2ec2a606a4f041bfa45569700f97df556f10779d/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/table/AsyncDynamicTableSinkFactory.java#L57)
 it seems you are reusing some properties here



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/format/AbstractRedshiftOutputFormat.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.redshift.format;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.connector.redshift.connection.RedshiftConnectionProvider;
+import org.apache.flink.connector.redshift.executor.RedshiftExecutor;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Flushable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/** Abstract Redshift Output format. */
+@Internal
+public abstract class AbstractRedshiftOutputFormat extends 
RichOutputFormat<RowData>
+        implements Flushable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractRedshiftOutputFormat.class);
+
+    protected transient volatile boolean closed = false;

Review Comment:
   this is smelly, have you tested that away from local clusters and with 
checkpointing? 



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableFactory.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.redshift.executor.RedshiftS3Util;
+import org.apache.flink.connector.redshift.mode.SinkMode;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/** Dynamic Table Factory. */
+@PublicEvolving
+public class RedshiftDynamicTableFactory implements DynamicTableSinkFactory {
+    public static final String IDENTIFIER = "redshift";
+
+    public static final ConfigOption<String> HOSTNAME =
+            ConfigOptions.key("hostname")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDeprecatedKeys("AWS Redshift cluster hostname.");
+
+    public static final ConfigOption<Integer> PORT =
+            ConfigOptions.key("port")
+                    .intType()
+                    .defaultValue(5439)
+                    .withDeprecatedKeys("AWS Redshift port number.\nDefault 
value : 5439.");
+
+    public static final ConfigOption<String> USERNAME =
+            ConfigOptions.key("username")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift Cluster username.");
+
+    public static final ConfigOption<String> PASSWORD =
+            ConfigOptions.key("password")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster password.");
+
+    public static final ConfigOption<String> DATABASE_NAME =
+            ConfigOptions.key("sink.database-name")
+                    .stringType()
+                    .defaultValue("dev")
+                    .withDescription(
+                            "AWS Redshift cluster database name. Default value 
set to `dev`.");
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("sink.table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster sink table name.");
+
+    public static final ConfigOption<Integer> SINK_BATCH_SIZE =
+            ConfigOptions.key("sink.batch-size")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "`sink.batch-size` determines the maximum size of 
batch, in terms of the number of records, "
+                                    + "at which data will trigger a flush 
operation."
+                                    + " When the number of records exceeds 
this threshold, the system initiates a flush to manage the data.\n"
+                                    + "Default Value: 1000");
+
+    public static final ConfigOption<Duration> SINK_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.flush-interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1L))
+                    .withDescription(
+                            "the flush interval mills, over this time, 
asynchronous threads will flush data. The default value is 1s.");
+
+    public static final ConfigOption<Integer> SINK_MAX_RETRIES =
+            ConfigOptions.key("sink.max-retries")
+                    .intType()
+                    .defaultValue(3)
+                    .withDescription("the max retry times if writing records 
to database failed.");
+
+    public static final ConfigOption<SinkMode> SINK_MODE =
+            ConfigOptions.key("sink.mode")
+                    .enumType(SinkMode.class)
+                    .noDefaultValue()
+                    .withDescription(
+                            "Currently, 2 modes are supported for Flink 
connector redshift.\n"
+                                    + "\t 1) COPY Mode."
+                                    + "\t 2) JDBC Mode.");
+    public static final ConfigOption<String> TEMP_S3_URI =
+            ConfigOptions.key("sink.copy-mode.aws.s3-uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("using Redshift COPY command must provide 
a S3 URI.");
+    public static final ConfigOption<String> IAM_ROLE_ARN =

Review Comment:
   Should we use the [existing aws authentication 
way](https://github.com/apache/flink-connector-aws/blob/b512749f53140f8d6e1695cdeb2b6be2d5c40c50/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/AWSOptionUtils.java#L36)?
 



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableSink.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.redshift.format.AbstractRedshiftOutputFormat;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.OutputFormatProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+
+/** AWS Redshift Dynamic Table Sink . */
+@PublicEvolving
+public class RedshiftDynamicTableSink implements DynamicTableSink {
+    private final String[] primaryKeys;
+
+    private final String[] fieldNames;
+
+    private final DataType[] fieldDataTypes;
+
+    private final RedshiftOptions options;
+
+    public RedshiftDynamicTableSink(
+            RedshiftOptions options,
+            String[] primaryKeys,
+            String[] fieldNames,
+            DataType[] fieldDataTypes) {
+

Review Comment:
   nit: unnecessary new line



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableFactory.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.redshift.executor.RedshiftS3Util;
+import org.apache.flink.connector.redshift.mode.SinkMode;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/** Dynamic Table Factory. */
+@PublicEvolving
+public class RedshiftDynamicTableFactory implements DynamicTableSinkFactory {
+    public static final String IDENTIFIER = "redshift";
+
+    public static final ConfigOption<String> HOSTNAME =
+            ConfigOptions.key("hostname")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDeprecatedKeys("AWS Redshift cluster hostname.");
+
+    public static final ConfigOption<Integer> PORT =
+            ConfigOptions.key("port")
+                    .intType()
+                    .defaultValue(5439)
+                    .withDeprecatedKeys("AWS Redshift port number.\nDefault 
value : 5439.");
+
+    public static final ConfigOption<String> USERNAME =
+            ConfigOptions.key("username")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift Cluster username.");
+
+    public static final ConfigOption<String> PASSWORD =
+            ConfigOptions.key("password")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster password.");
+
+    public static final ConfigOption<String> DATABASE_NAME =
+            ConfigOptions.key("sink.database-name")
+                    .stringType()
+                    .defaultValue("dev")

Review Comment:
   Why do we need to set that?



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableFactory.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.redshift.executor.RedshiftS3Util;
+import org.apache.flink.connector.redshift.mode.SinkMode;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/** Dynamic Table Factory. */
+@PublicEvolving
+public class RedshiftDynamicTableFactory implements DynamicTableSinkFactory {
+    public static final String IDENTIFIER = "redshift";

Review Comment:
   nit: I would move the configs to a separate file as in 
[flink-connector-aws-kinesis-streams](https://github.com/apache/flink-connector-aws/blob/main/flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/table/KinesisConnectorOptions.java)



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableSink.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.redshift.format.AbstractRedshiftOutputFormat;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.OutputFormatProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+
+/** AWS Redshift Dynamic Table Sink . */
+@PublicEvolving
+public class RedshiftDynamicTableSink implements DynamicTableSink {
+    private final String[] primaryKeys;
+
+    private final String[] fieldNames;
+
+    private final DataType[] fieldDataTypes;
+
+    private final RedshiftOptions options;
+
+    public RedshiftDynamicTableSink(
+            RedshiftOptions options,
+            String[] primaryKeys,
+            String[] fieldNames,
+            DataType[] fieldDataTypes) {
+
+        this.primaryKeys = primaryKeys;
+        this.fieldNames = fieldNames;
+        this.fieldDataTypes = fieldDataTypes;
+        this.options = options;
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        validatePrimaryKey(requestedMode);
+        return ChangelogMode.newBuilder()
+                .addContainedKind(RowKind.INSERT)
+                .addContainedKind(RowKind.UPDATE_AFTER)
+                .addContainedKind(RowKind.DELETE)
+                .build();
+    }
+
+    private void validatePrimaryKey(ChangelogMode requestedMode) {
+        Preconditions.checkState(
+                (ChangelogMode.insertOnly().equals(requestedMode) || 
primaryKeys.length > 0),
+                "Declare primary key for UPSERT operation.");
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+

Review Comment:
   nit: remove newline



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableSink.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.redshift.format.AbstractRedshiftOutputFormat;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.OutputFormatProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+
+/** AWS Redshift Dynamic Table Sink . */
+@PublicEvolving
+public class RedshiftDynamicTableSink implements DynamicTableSink {
+    private final String[] primaryKeys;
+
+    private final String[] fieldNames;
+
+    private final DataType[] fieldDataTypes;
+
+    private final RedshiftOptions options;
+
+    public RedshiftDynamicTableSink(
+            RedshiftOptions options,
+            String[] primaryKeys,
+            String[] fieldNames,
+            DataType[] fieldDataTypes) {
+
+        this.primaryKeys = primaryKeys;
+        this.fieldNames = fieldNames;
+        this.fieldDataTypes = fieldDataTypes;
+        this.options = options;
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        validatePrimaryKey(requestedMode);
+        return ChangelogMode.newBuilder()
+                .addContainedKind(RowKind.INSERT)
+                .addContainedKind(RowKind.UPDATE_AFTER)
+                .addContainedKind(RowKind.DELETE)
+                .build();

Review Comment:
   We should use `ChangelogMode.upsert()` 



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableFactory.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.redshift.executor.RedshiftS3Util;
+import org.apache.flink.connector.redshift.mode.SinkMode;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.table.types.DataType;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+/** Dynamic Table Factory. */
+@PublicEvolving
+public class RedshiftDynamicTableFactory implements DynamicTableSinkFactory {
+    public static final String IDENTIFIER = "redshift";
+
+    public static final ConfigOption<String> HOSTNAME =
+            ConfigOptions.key("hostname")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDeprecatedKeys("AWS Redshift cluster hostname.");
+
+    public static final ConfigOption<Integer> PORT =
+            ConfigOptions.key("port")
+                    .intType()
+                    .defaultValue(5439)
+                    .withDeprecatedKeys("AWS Redshift port number.\nDefault 
value : 5439.");
+
+    public static final ConfigOption<String> USERNAME =
+            ConfigOptions.key("username")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift Cluster username.");
+
+    public static final ConfigOption<String> PASSWORD =
+            ConfigOptions.key("password")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster password.");
+
+    public static final ConfigOption<String> DATABASE_NAME =
+            ConfigOptions.key("sink.database-name")
+                    .stringType()
+                    .defaultValue("dev")
+                    .withDescription(
+                            "AWS Redshift cluster database name. Default value 
set to `dev`.");
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("sink.table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("AWS Redshift cluster sink table name.");
+
+    public static final ConfigOption<Integer> SINK_BATCH_SIZE =
+            ConfigOptions.key("sink.batch-size")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "`sink.batch-size` determines the maximum size of 
batch, in terms of the number of records, "
+                                    + "at which data will trigger a flush 
operation."
+                                    + " When the number of records exceeds 
this threshold, the system initiates a flush to manage the data.\n"
+                                    + "Default Value: 1000");
+
+    public static final ConfigOption<Duration> SINK_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.flush-interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1L))
+                    .withDescription(
+                            "the flush interval mills, over this time, 
asynchronous threads will flush data. The default value is 1s.");
+
+    public static final ConfigOption<Integer> SINK_MAX_RETRIES =
+            ConfigOptions.key("sink.max-retries")
+                    .intType()
+                    .defaultValue(3)
+                    .withDescription("the max retry times if writing records 
to database failed.");
+
+    public static final ConfigOption<SinkMode> SINK_MODE =
+            ConfigOptions.key("sink.mode")
+                    .enumType(SinkMode.class)
+                    .noDefaultValue()
+                    .withDescription(
+                            "Currently, 2 modes are supported for Flink 
connector redshift.\n"
+                                    + "\t 1) COPY Mode."
+                                    + "\t 2) JDBC Mode.");
+    public static final ConfigOption<String> TEMP_S3_URI =

Review Comment:
   Why `TEMP`?



##########
flink-connector-aws/flink-connector-redshift/src/main/resources/META-INF/services/.gitkeep:
##########


Review Comment:
   Why is this needed?



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/table/RedshiftDynamicTableSink.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.redshift.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.redshift.format.AbstractRedshiftOutputFormat;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.OutputFormatProvider;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.Preconditions;
+
+/** AWS Redshift Dynamic Table Sink . */
+@PublicEvolving
+public class RedshiftDynamicTableSink implements DynamicTableSink {
+    private final String[] primaryKeys;
+
+    private final String[] fieldNames;
+
+    private final DataType[] fieldDataTypes;
+
+    private final RedshiftOptions options;
+
+    public RedshiftDynamicTableSink(
+            RedshiftOptions options,
+            String[] primaryKeys,
+            String[] fieldNames,
+            DataType[] fieldDataTypes) {
+
+        this.primaryKeys = primaryKeys;
+        this.fieldNames = fieldNames;
+        this.fieldDataTypes = fieldDataTypes;
+        this.options = options;
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        validatePrimaryKey(requestedMode);

Review Comment:
   Why do we need a validation here?



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/format/AbstractRedshiftOutputFormat.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.redshift.format;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.connector.redshift.connection.RedshiftConnectionProvider;
+import org.apache.flink.connector.redshift.executor.RedshiftExecutor;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Flushable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/** Abstract Redshift Output format. */
+@Internal
+public abstract class AbstractRedshiftOutputFormat extends 
RichOutputFormat<RowData>
+        implements Flushable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractRedshiftOutputFormat.class);
+
+    protected transient volatile boolean closed = false;
+
+    protected transient ScheduledExecutorService scheduler;
+
+    protected transient ScheduledFuture<?> scheduledFuture;
+
+    protected transient volatile Exception flushException;
+
+    public AbstractRedshiftOutputFormat() {}
+
+    @Override
+    public void configure(Configuration parameters) {}
+
+    public void scheduledFlush(long intervalMillis, String executorName) {
+        Preconditions.checkArgument(intervalMillis > 0, "flush interval must 
be greater than 0");
+        scheduler = new ScheduledThreadPoolExecutor(1, new 
ExecutorThreadFactory(executorName));
+        scheduledFuture =
+                scheduler.scheduleWithFixedDelay(
+                        () -> {
+                            synchronized (this) {
+                                if (!closed) {
+                                    try {
+                                        flush();
+                                    } catch (Exception e) {
+                                        flushException = e;
+                                    }
+                                }
+                            }
+                        },
+                        intervalMillis,
+                        intervalMillis,
+                        TimeUnit.MILLISECONDS);
+    }
+
+    public void checkBeforeFlush(final RedshiftExecutor executor) throws 
IOException {
+        checkFlushException();
+        try {
+            executor.executeBatch();
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public synchronized void close() {
+        if (!closed) {
+            closed = true;
+
+            try {
+                flush();
+            } catch (Exception exception) {
+                LOG.warn("Flushing records to Redshift failed.", exception);

Review Comment:
   We are swallowing all exceptions here, this seems like a smell and could 
possibly break delivery guarantees. We should capture specific exceptions only 
and bubble/wrap up the rest.



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/options/RedshiftOptions.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.redshift.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.redshift.mode.SinkMode;
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Optional;
+
+/** Options. */

Review Comment:
   nit: Could we use more descriptive Javadoc for example: "Options to 
configure connection to redshift"



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/format/AbstractRedshiftOutputFormat.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.redshift.format;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.connector.redshift.connection.RedshiftConnectionProvider;
+import org.apache.flink.connector.redshift.executor.RedshiftExecutor;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Flushable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/** Abstract Redshift Output format. */
+@Internal
+public abstract class AbstractRedshiftOutputFormat extends 
RichOutputFormat<RowData>
+        implements Flushable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractRedshiftOutputFormat.class);
+
+    protected transient volatile boolean closed = false;
+
+    protected transient ScheduledExecutorService scheduler;
+
+    protected transient ScheduledFuture<?> scheduledFuture;
+
+    protected transient volatile Exception flushException;
+
+    public AbstractRedshiftOutputFormat() {}
+
+    @Override
+    public void configure(Configuration parameters) {}
+
+    public void scheduledFlush(long intervalMillis, String executorName) {
+        Preconditions.checkArgument(intervalMillis > 0, "flush interval must 
be greater than 0");
+        scheduler = new ScheduledThreadPoolExecutor(1, new 
ExecutorThreadFactory(executorName));
+        scheduledFuture =
+                scheduler.scheduleWithFixedDelay(
+                        () -> {
+                            synchronized (this) {
+                                if (!closed) {
+                                    try {
+                                        flush();
+                                    } catch (Exception e) {
+                                        flushException = e;
+                                    }
+                                }
+                            }
+                        },
+                        intervalMillis,
+                        intervalMillis,
+                        TimeUnit.MILLISECONDS);
+    }
+
+    public void checkBeforeFlush(final RedshiftExecutor executor) throws 
IOException {
+        checkFlushException();
+        try {
+            executor.executeBatch();
+        } catch (Exception e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public synchronized void close() {
+        if (!closed) {
+            closed = true;
+

Review Comment:
   nit: remove new line



##########
flink-connector-aws/flink-connector-redshift/src/main/java/org/apache/flink/connector/redshift/format/AbstractRedshiftOutputFormat.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.redshift.format;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.io.RichOutputFormat;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.connector.redshift.connection.RedshiftConnectionProvider;
+import org.apache.flink.connector.redshift.executor.RedshiftExecutor;
+import org.apache.flink.connector.redshift.options.RedshiftOptions;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.concurrent.ExecutorThreadFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Flushable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/** Abstract Redshift Output format. */
+@Internal
+public abstract class AbstractRedshiftOutputFormat extends 
RichOutputFormat<RowData>
+        implements Flushable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractRedshiftOutputFormat.class);
+
+    protected transient volatile boolean closed = false;
+
+    protected transient ScheduledExecutorService scheduler;
+
+    protected transient ScheduledFuture<?> scheduledFuture;
+
+    protected transient volatile Exception flushException;
+
+    public AbstractRedshiftOutputFormat() {}
+
+    @Override
+    public void configure(Configuration parameters) {}
+
+    public void scheduledFlush(long intervalMillis, String executorName) {
+        Preconditions.checkArgument(intervalMillis > 0, "flush interval must 
be greater than 0");
+        scheduler = new ScheduledThreadPoolExecutor(1, new 
ExecutorThreadFactory(executorName));
+        scheduledFuture =

Review Comment:
   This breaks the execution model, You should use the `mailboxExecutor` 
instead. 



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