hlteoh37 commented on code in PR #155:
URL: 
https://github.com/apache/flink-connector-aws/pull/155#discussion_r1755166214


##########
flink-connector-aws/flink-connector-sqs/src/main/java/org/apache/flink/connector/sqs/table/SqsDynamicSink.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.sqs.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSink;
+import org.apache.flink.connector.base.table.sink.AsyncDynamicTableSinkBuilder;
+import org.apache.flink.connector.sqs.sink.SqsSink;
+import org.apache.flink.connector.sqs.sink.SqsSinkBuilder;
+import org.apache.flink.connector.sqs.sink.SqsSinkElementConverter;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+
+import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+/** A {@link DynamicTableSink} for SQS. */
+@Internal
+public class SqsDynamicSink extends 
AsyncDynamicTableSink<SendMessageBatchRequestEntry> {
+
+    /** Consumed data type of the table. */
+    private final DataType consumedDataType;
+
+    /** Url of Sqs queue to write to. */
+    private final String sqsUrl;
+
+    /** Properties for the Sqs Aws Client. */
+    private final Properties sqsClientProps;
+
+    /** Encoding format to convert between row data and byte array. */
+    EncodingFormat<SerializationSchema<RowData>> encodingFormat;
+
+    /** Flag to determine whether to fail on error. */
+    private final Boolean failOnError;
+
+    protected SqsDynamicSink(
+            @Nullable Integer maxBatchSize,
+            @Nullable Integer maxInFlightRequests,
+            @Nullable Integer maxBufferedRequests,
+            @Nullable Long maxBufferSizeInBytes,
+            @Nullable Long maxTimeInBufferMS,
+            @Nullable Boolean failOnError,
+            @Nullable DataType consumedDataType,
+            EncodingFormat<SerializationSchema<RowData>> encodingFormat,
+            String sqsUrl,

Review Comment:
   can we add some `Preconditions` checks on this please?



##########
flink-connector-aws/flink-connector-sqs/src/main/java/org/apache/flink/connector/sqs/table/SqsConnectorOptions.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.sqs.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+import java.util.Map;
+
+/** Options for the SQS connector. */
+@PublicEvolving
+public class SqsConnectorOptions {

Review Comment:
   We havent' released the connector, so let's make this uniform before we 
release it



##########
flink-connector-aws/flink-connector-sqs/src/main/java/org/apache/flink/connector/sqs/table/SqsConnectorOptions.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.sqs.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+import java.util.Map;
+
+/** Options for the SQS connector. */
+@PublicEvolving
+public class SqsConnectorOptions {

Review Comment:
   Can we merge this with `SqsConfigConstants`?



##########
flink-connector-aws/flink-connector-sqs/pom.xml:
##########
@@ -103,6 +103,28 @@ under the License.
             <artifactId>jackson-datatype-jsr310</artifactId>
         </dependency>
 
+        <!-- Table API dependencies-->
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-common</artifactId>
+            <version>${flink.version}</version>
+        </dependency>

Review Comment:
   can we make this `provided`?



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