ic4y commented on code in PR #3309:
URL: 
https://github.com/apache/incubator-seatunnel/pull/3309#discussion_r1019784321


##########
seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkClient.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.seatunnel.connectors.seatunnel.tablestore.sink;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.utils.SeaTunnelException;
+import 
org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions;
+
+import com.alicloud.openservices.tablestore.SyncClient;
+import com.alicloud.openservices.tablestore.model.BatchWriteRowRequest;
+import com.alicloud.openservices.tablestore.model.BatchWriteRowResponse;
+import com.alicloud.openservices.tablestore.model.RowPutChange;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+@Slf4j
+public class TablestoreSinkClient {
+    private final TablestoreOptions tablestoreOptions;
+    private ScheduledExecutorService scheduler;
+    private ScheduledFuture<?> scheduledFuture;
+    private volatile boolean initialize;
+    private volatile Exception flushException;
+    private SyncClient syncClient;
+    private final List<RowPutChange> batchList;
+
+    public TablestoreSinkClient(TablestoreOptions tablestoreOptions, 
SeaTunnelRowType typeInfo) {
+        this.tablestoreOptions = tablestoreOptions;
+        this.batchList = new ArrayList<>();
+    }
+
+    private void tryInit() throws IOException {
+        if (initialize) {
+            return;
+        }
+        syncClient = new SyncClient(
+            tablestoreOptions.getEndpoint(),
+            tablestoreOptions.getAccessKeyId(),
+            tablestoreOptions.getAccessKeySecret(),
+            tablestoreOptions.getInstanceName());
+
+        scheduler = Executors.newSingleThreadScheduledExecutor(
+            new 
ThreadFactoryBuilder().setNameFormat("Tablestore-sink-output-%s").build());
+        scheduledFuture = scheduler.scheduleAtFixedRate(
+            () -> {
+                try {
+                    flush();
+                } catch (IOException e) {
+                    flushException = e;
+                }
+            },
+            tablestoreOptions.getBatchIntervalMs(),
+            tablestoreOptions.getBatchIntervalMs(),
+            TimeUnit.MILLISECONDS);
+
+        initialize = true;
+    }
+
+    public synchronized void write(RowPutChange rowPutChange) throws 
IOException {

Review Comment:
   no synchronized needed



##########
seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java:
##########
@@ -139,7 +130,7 @@ private SeaTunnelRowType initTableField(Connection conn) {
         ArrayList<String> fieldNames = new ArrayList<>();
         try {
             PreparedStatement ps = 
conn.prepareStatement(jdbcSourceOptions.getJdbcConnectionOptions().getQuery());
-            ResultSetMetaData resultSetMetaData = ps.getMetaData();
+            ResultSetMetaData resultSetMetaData = 
ps.executeQuery().getMetaData();

Review Comment:
   why need add executeQuery



##########
seatunnel-dist/pom.xml:
##########
@@ -321,6 +321,11 @@
                     <version>${project.version}</version>
                     <scope>provided</scope>
                 </dependency>
+                <dependency>
+                    <groupId>org.apache.seatunnel</groupId>
+                    <artifactId>connector-tablestore</artifactId>
+                    <version>${project.version}</version>
+                    <scope>provided</scope>                </dependency>

Review Comment:
   keep the format



##########
seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/config/TablestoreOptions.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.seatunnel.connectors.seatunnel.tablestore.config;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+
+import java.io.Serializable;
+import java.util.List;
+
+@Data
+@AllArgsConstructor
+public class TablestoreOptions implements Serializable {
+
+    private static final int DEFAULT_BATCH_SIZE = 25;
+    private static final int DEFAULT_BATCH_INTERVAL_MS = 1000;
+
+    private String endpoint;
+
+    private String instanceName;
+
+    private String accessKeyId;
+
+    private String accessKeySecret;
+
+    private String table;
+
+    private List<String> primaryKeys;
+
+    public int batchSize = DEFAULT_BATCH_SIZE;
+    public int batchIntervalMs = DEFAULT_BATCH_INTERVAL_MS;
+
+    public TablestoreOptions(Config config) {
+        this.endpoint = config.getString(TablestoreConfig.END_POINT);
+        this.instanceName = config.getString(TablestoreConfig.INSTANCE_NAME);
+        this.accessKeyId = config.getString(TablestoreConfig.ACCESS_KEY_ID);
+        this.accessKeySecret = 
config.getString(TablestoreConfig.ACCESS_KEY_SECRET);
+        this.table = config.getString(TablestoreConfig.TABLE);
+
+        if (config.hasPath(TablestoreConfig.BATCH_SIZE)) {
+            this.batchSize = config.getInt(TablestoreConfig.BATCH_SIZE);
+        }
+        if (config.hasPath(TablestoreConfig.DEFAULT_BATCH_INTERVAL_MS)) {
+            this.batchIntervalMs = 
config.getInt(TablestoreConfig.DEFAULT_BATCH_INTERVAL_MS);
+        }
+        if (config.hasPath(TablestoreConfig.PRIMARY_KEYS)) {

Review Comment:
   already checked in checkAllExists



##########
seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkClient.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.seatunnel.connectors.seatunnel.tablestore.sink;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.utils.SeaTunnelException;
+import 
org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions;
+
+import com.alicloud.openservices.tablestore.SyncClient;
+import com.alicloud.openservices.tablestore.model.BatchWriteRowRequest;
+import com.alicloud.openservices.tablestore.model.BatchWriteRowResponse;
+import com.alicloud.openservices.tablestore.model.RowPutChange;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+@Slf4j
+public class TablestoreSinkClient {
+    private final TablestoreOptions tablestoreOptions;
+    private ScheduledExecutorService scheduler;
+    private ScheduledFuture<?> scheduledFuture;
+    private volatile boolean initialize;
+    private volatile Exception flushException;
+    private SyncClient syncClient;
+    private final List<RowPutChange> batchList;
+
+    public TablestoreSinkClient(TablestoreOptions tablestoreOptions, 
SeaTunnelRowType typeInfo) {
+        this.tablestoreOptions = tablestoreOptions;
+        this.batchList = new ArrayList<>();
+    }
+
+    private void tryInit() throws IOException {
+        if (initialize) {
+            return;
+        }
+        syncClient = new SyncClient(
+            tablestoreOptions.getEndpoint(),
+            tablestoreOptions.getAccessKeyId(),
+            tablestoreOptions.getAccessKeySecret(),
+            tablestoreOptions.getInstanceName());
+
+        scheduler = Executors.newSingleThreadScheduledExecutor(
+            new 
ThreadFactoryBuilder().setNameFormat("Tablestore-sink-output-%s").build());
+        scheduledFuture = scheduler.scheduleAtFixedRate(
+            () -> {
+                try {
+                    flush();
+                } catch (IOException e) {
+                    flushException = e;
+                }
+            },
+            tablestoreOptions.getBatchIntervalMs(),
+            tablestoreOptions.getBatchIntervalMs(),
+            TimeUnit.MILLISECONDS);
+
+        initialize = true;
+    }
+
+    public synchronized void write(RowPutChange rowPutChange) throws 
IOException {
+        tryInit();
+        checkFlushException();
+        batchList.add(rowPutChange);
+        if (tablestoreOptions.getBatchSize() > 0
+            && batchList.size() >= tablestoreOptions.getBatchSize()) {
+            flush();
+        }
+    }
+
+    public synchronized void close() throws IOException {

Review Comment:
   Ditto



##########
seatunnel-connectors-v2/connector-tablestore/src/main/java/org/apache/seatunnel/connectors/seatunnel/tablestore/sink/TablestoreSinkClient.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.seatunnel.connectors.seatunnel.tablestore.sink;
+
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.utils.SeaTunnelException;
+import 
org.apache.seatunnel.connectors.seatunnel.tablestore.config.TablestoreOptions;
+
+import com.alicloud.openservices.tablestore.SyncClient;
+import com.alicloud.openservices.tablestore.model.BatchWriteRowRequest;
+import com.alicloud.openservices.tablestore.model.BatchWriteRowResponse;
+import com.alicloud.openservices.tablestore.model.RowPutChange;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+@Slf4j
+public class TablestoreSinkClient {
+    private final TablestoreOptions tablestoreOptions;
+    private ScheduledExecutorService scheduler;
+    private ScheduledFuture<?> scheduledFuture;
+    private volatile boolean initialize;
+    private volatile Exception flushException;
+    private SyncClient syncClient;
+    private final List<RowPutChange> batchList;
+
+    public TablestoreSinkClient(TablestoreOptions tablestoreOptions, 
SeaTunnelRowType typeInfo) {
+        this.tablestoreOptions = tablestoreOptions;
+        this.batchList = new ArrayList<>();
+    }
+
+    private void tryInit() throws IOException {
+        if (initialize) {
+            return;
+        }
+        syncClient = new SyncClient(
+            tablestoreOptions.getEndpoint(),
+            tablestoreOptions.getAccessKeyId(),
+            tablestoreOptions.getAccessKeySecret(),
+            tablestoreOptions.getInstanceName());
+
+        scheduler = Executors.newSingleThreadScheduledExecutor(
+            new 
ThreadFactoryBuilder().setNameFormat("Tablestore-sink-output-%s").build());
+        scheduledFuture = scheduler.scheduleAtFixedRate(
+            () -> {
+                try {
+                    flush();
+                } catch (IOException e) {
+                    flushException = e;
+                }
+            },
+            tablestoreOptions.getBatchIntervalMs(),
+            tablestoreOptions.getBatchIntervalMs(),
+            TimeUnit.MILLISECONDS);
+
+        initialize = true;
+    }
+
+    public synchronized void write(RowPutChange rowPutChange) throws 
IOException {
+        tryInit();
+        checkFlushException();
+        batchList.add(rowPutChange);
+        if (tablestoreOptions.getBatchSize() > 0
+            && batchList.size() >= tablestoreOptions.getBatchSize()) {
+            flush();
+        }
+    }
+
+    public synchronized void close() throws IOException {
+        if (scheduledFuture != null) {
+            scheduledFuture.cancel(false);
+            scheduler.shutdown();
+        }
+        if (syncClient != null) {
+            flush();
+            syncClient.shutdown();
+        }
+    }
+
+    synchronized void flush() throws IOException {

Review Comment:
   Ditto



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

Reply via email to