Hisoka-X commented on code in PR #9661:
URL: https://github.com/apache/seatunnel/pull/9661#discussion_r2287537327


##########
docs/en/connector-v2/sink/Databend.md:
##########
@@ -37,7 +37,7 @@ The Databend sink internally implements bulk data import 
through stage attachmen
 | Name | Type | Required | Default Value | Description                         
        |
 
|------|------|----------|---------------|---------------------------------------------|
 | url | String | Yes | - | Databend JDBC connection URL               |
-| username | String | Yes | - | Databend database username                    |
+| user | String | Yes | - | Databend database username                    |

Review Comment:
   let's rollback to `username`, there are some mistakes in doc before 
https://github.com/apache/seatunnel/pull/9679.



##########
seatunnel-connectors-v2/connector-databend/src/main/java/org/apache/seatunnel/connectors/seatunnel/databend/config/DatabendSinkOptions.java:
##########
@@ -47,4 +47,28 @@ public class DatabendSinkOptions {
                     .intType()
                     .defaultValue(300)
                     .withDescription("The timeout seconds for Databend client 
execution");
+
+    public static final Option<Integer> BATCH_SIZE =
+            Options.key("batch_size")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription("Batch size for CDC merge operations");
+
+    public static final Option<Integer> INTERVAL =

Review Comment:
   Do we still need this config?



##########
seatunnel-connectors-v2/connector-databend/src/main/java/org/apache/seatunnel/connectors/seatunnel/databend/sink/DatabendSinkAggregatedCommitter.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.databend.sink;
+
+import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
+import org.apache.seatunnel.api.table.catalog.CatalogTable;
+import 
org.apache.seatunnel.connectors.seatunnel.databend.config.DatabendSinkConfig;
+import 
org.apache.seatunnel.connectors.seatunnel.databend.exception.DatabendConnectorErrorCode;
+import 
org.apache.seatunnel.connectors.seatunnel.databend.exception.DatabendConnectorException;
+import org.apache.seatunnel.connectors.seatunnel.databend.util.DatabendUtil;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Aggregated committer for Databend sink that handles CDC (Change Data 
Capture) operations. In CDC
+ * mode, this committer performs merge operations to apply changes to the 
target table. Merge
+ * operations are only performed when the accumulated record count reaches the 
configured batch
+ * size, which helps optimize performance by reducing the frequency of merge 
operations.
+ */
+@Slf4j
+public class DatabendSinkAggregatedCommitter
+        implements SinkAggregatedCommitter<
+                DatabendSinkCommitterInfo, DatabendSinkAggregatedCommitInfo> {
+
+    // Add a unique identifier for each instance
+    private static final AtomicLong INSTANCE_COUNTER = new AtomicLong(0);
+    private final long instanceId = INSTANCE_COUNTER.getAndIncrement();
+
+    private final DatabendSinkConfig databendSinkConfig;
+    private final String database;
+    private final String table;
+    private final String rawTableName;
+    private final String streamName;
+
+    private Connection connection;
+    private boolean isCdcMode;
+    // Store catalog table to access schema information
+    private CatalogTable catalogTable;
+
+    // Add a setter for catalogTable
+    public void setCatalogTable(CatalogTable catalogTable) {
+        this.catalogTable = catalogTable;
+    }
+
+    public DatabendSinkAggregatedCommitter(
+            DatabendSinkConfig databendSinkConfig,
+            String database,
+            String table,
+            String rawTableName,
+            String streamName) {
+        this.databendSinkConfig = databendSinkConfig;
+        this.database = database;
+        this.table = table;
+        this.rawTableName = rawTableName;
+        this.streamName = streamName;
+        this.isCdcMode = databendSinkConfig.isCdcMode();
+    }
+
+    @Override
+    public void init() {
+        try {
+            log.info("[Instance {}] Initializing 
DatabendSinkAggregatedCommitter", instanceId);
+            log.info("[Instance {}] DatabendSinkConfig: {}", instanceId, 
databendSinkConfig);
+            log.info("[Instance {}] Database: {}", instanceId, database);
+            log.info("[Instance {}] Table: {}", instanceId, table);
+            log.info("[Instance {}] Is CDC mode: {}", instanceId, isCdcMode);
+
+            this.connection = 
DatabendUtil.createConnection(databendSinkConfig);
+            log.info(
+                    "[Instance {}] Databend connection created successfully: 
{}",
+                    instanceId,
+                    connection);
+
+            // CDC infrastructure is now initialized in 
DatabendSink.setJobContext
+            // Just log that we're in CDC mode
+            if (isCdcMode) {
+                log.info("[Instance {}] Running in CDC mode", instanceId);
+            }
+        } catch (SQLException e) {
+            log.error(
+                    "[Instance {}] Failed to initialize 
DatabendSinkAggregatedCommitter: {}",
+                    instanceId,
+                    e.getMessage(),
+                    e);
+            throw new DatabendConnectorException(
+                    DatabendConnectorErrorCode.CONNECT_FAILED,
+                    "Failed to initialize DatabendSinkAggregatedCommitter: " + 
e.getMessage(),
+                    e);
+        } catch (Exception e) {
+            log.error(
+                    "[Instance {}] Unexpected error during initialization: {}",
+                    instanceId,
+                    e.getMessage(),
+                    e);
+            throw e;
+        }
+    }
+
+    private String getCurrentTimestamp() {
+        return 
LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyyMMddHHmmssSSS"));
+    }
+
+    @Override
+    public List<DatabendSinkAggregatedCommitInfo> commit(
+            List<DatabendSinkAggregatedCommitInfo> aggregatedCommitInfos) 
throws IOException {
+        // Perform final merge operation in CDC mode only when necessary
+        if (isCdcMode && shouldPerformMerge()) {

Review Comment:
   ```suggestion
           if (isCdcMode) {
   ```
   
   We should execute merge when commit method be invoked every times. The 
interval of this method be invoked equals `checkpoint.interval`



##########
seatunnel-connectors-v2/connector-databend/src/main/java/org/apache/seatunnel/connectors/seatunnel/databend/sink/DatabendSink.java:
##########
@@ -220,4 +247,109 @@ private String convertToDatabendType(SeaTunnelDataType<?> 
dataType) {
                 return "STRING"; // Default to STRING for complex types
         }
     }
+
+    @Override
+    public Optional<
+                    SinkAggregatedCommitter<
+                            DatabendSinkCommitterInfo, 
DatabendSinkAggregatedCommitInfo>>
+            createAggregatedCommitter() throws IOException {
+        DatabendSinkAggregatedCommitter committer =
+                new DatabendSinkAggregatedCommitter(
+                        databendSinkConfig, database, table, rawTableName, 
streamName);
+        committer.setCatalogTable(catalogTable);
+        return Optional.of(committer);
+    }
+
+    @Override
+    public Optional<Serializer<DatabendSinkCommitterInfo>> 
getCommitInfoSerializer() {
+        return Optional.of(new DefaultSerializer<>());
+    }
+
+    @Override
+    public Optional<Serializer<DatabendSinkAggregatedCommitInfo>>
+            getAggregatedCommitInfoSerializer() {
+        return Optional.of(new DefaultSerializer<>());
+    }
+
+    @Override
+    public Optional<CatalogTable> getWriteCatalogTable() {
+        return SeaTunnelSink.super.getWriteCatalogTable();
+    }

Review Comment:
   ditto



##########
seatunnel-connectors-v2/connector-databend/src/main/java/org/apache/seatunnel/connectors/seatunnel/databend/sink/DatabendSink.java:
##########
@@ -174,6 +196,11 @@ public Optional<SaveModeHandler> getSaveModeHandler() {
         }
     }
 
+    @Override
+    public Optional<Serializer<Void>> getWriterStateSerializer() {
+        return Optional.empty();
+    }

Review Comment:
   The override seem useless because the behavior not changed.



##########
seatunnel-connectors-v2/connector-databend/src/main/java/org/apache/seatunnel/connectors/seatunnel/databend/config/DatabendSinkOptions.java:
##########
@@ -47,4 +47,28 @@ public class DatabendSinkOptions {
                     .intType()
                     .defaultValue(300)
                     .withDescription("The timeout seconds for Databend client 
execution");
+
+    public static final Option<Integer> BATCH_SIZE =
+            Options.key("batch_size")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription("Batch size for CDC merge operations");
+
+    public static final Option<Integer> INTERVAL =
+            Options.key("interval")
+                    .intType()
+                    .defaultValue(30)
+                    .withDescription("Interval in seconds for CDC merge 
operations");
+
+    public static final Option<String> CONFLICT_KEY =
+            Options.key("conflict_key")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Conflict key for CDC merge operations");
+
+    public static final Option<Boolean> ALLOW_DELETE =
+            Options.key("allow_delete")

Review Comment:
   @hantmac 



##########
seatunnel-connectors-v2/connector-databend/src/main/java/org/apache/seatunnel/connectors/seatunnel/databend/sink/DatabendSinkWriter.java:
##########
@@ -110,46 +143,178 @@ public DatabendSinkWriter(
                         e);
             }
         } else {
-            // use the catalog table schema to create the target table
-            SeaTunnelRowType rowType = catalogTable.getSeaTunnelRowType();
-            if (rowType == null || rowType.getFieldNames().length == 0) {
-                throw new DatabendConnectorException(
-                        DatabendConnectorErrorCode.SCHEMA_NOT_FOUND,
-                        "Source table schema is empty or null");
-            }
-
             try {
-                if (!tableExists(database, table)) {
+                if (isCdcMode) {
+                    // In CDC mode, we don't create tables here, it's done in 
AggregatedCommitter
+                    // We'll get the raw table and stream names from the 
committer via prepareCommit
                     log.info(
-                            "Target table {}.{} does not exist, creating with 
source schema",
-                            database,
-                            table);
-                    createTable(database, table, rowType);
+                            "CDC mode enabled, table creation will be handled 
by AggregatedCommitter");
                 } else {
-                    log.info("Target table {}.{} exists, verifying schema", 
database, table);
-                    verifyTableSchema(database, table, rowType);
+                    // Traditional mode
+                    initTraditionalMode(database, table);
                 }
             } catch (SQLException e) {
                 throw new DatabendConnectorException(
                         DatabendConnectorErrorCode.SQL_OPERATION_FAILED,
-                        "Failed to verify/create target table: " + 
e.getMessage(),
+                        "Failed to initialize sink writer: " + e.getMessage(),
                         e);
             }
+        }
+    }
 
-            this.insertSql = generateInsertSql(database, table, rowType);
-            log.info("Generated insert SQL: {}", insertSql);
-            try {
-                this.schemaChangeManager = new 
SchemaChangeManager(databendSinkConfig);
-                this.preparedStatement = 
connection.prepareStatement(insertSql);
-                this.preparedStatement.setQueryTimeout(executeTimeoutSec);
-                log.info("PreparedStatement created successfully");
-            } catch (SQLException e) {
-                throw new DatabendConnectorException(
-                        DatabendConnectorErrorCode.SQL_OPERATION_FAILED,
-                        "Failed to prepare statement: " + e.getMessage(),
-                        e);
-            }
+    private String getCurrentTimestamp() {
+        return 
LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyyMMddHHmmss"));
+    }
+
+    private void initializeCdcPreparedStatement() throws SQLException {
+        log.info("Initializing CDC PreparedStatement");
+
+        // In CDC mode, the rawTableName should be set by the 
AggregatedCommitter
+        // If it's not set yet, we can't proceed with CDC operations
+        if (rawTableName == null || rawTableName.isEmpty()) {
+            throw new DatabendConnectorException(
+                    DatabendConnectorErrorCode.SQL_OPERATION_FAILED,
+                    "Raw table name not set by AggregatedCommitter. Cannot 
initialize CDC PreparedStatement.");
+        }
+
+        // Generate insert SQL for raw table
+        String insertRawSql = 
generateInsertRawSql(sinkTablePath.getDatabaseName());
+
+        // Create the PreparedStatement
+        this.cdcPreparedStatement = connection.prepareStatement(insertRawSql);
+        this.cdcPreparedStatement.setQueryTimeout(executeTimeoutSec);
+
+        log.info("CDC PreparedStatement created successfully with SQL: {}", 
insertRawSql);
+    }
+
+    private void initTraditionalMode(String database, String table) throws 
SQLException {
+        // use the catalog table schema to create the target table
+        SeaTunnelRowType rowType = catalogTable.getSeaTunnelRowType();
+        if (rowType == null || rowType.getFieldNames().length == 0) {
+            throw new DatabendConnectorException(
+                    DatabendConnectorErrorCode.SCHEMA_NOT_FOUND,
+                    "Source table schema is empty or null");
+        }
+
+        if (!tableExists(database, table)) {
+            log.info(
+                    "Target table {}.{} does not exist, creating with source 
schema",
+                    database,
+                    table);
+            createTable(database, table, rowType);
+        } else {
+            log.info("Target table {}.{} exists, verifying schema", database, 
table);
+            verifyTableSchema(database, table, rowType);
+        }
+
+        this.insertSql = generateInsertSql(database, table, rowType);
+        log.info("Generated insert SQL: {}", insertSql);
+        try {
+            this.schemaChangeManager = new 
SchemaChangeManager(databendSinkConfig);
+            this.preparedStatement = connection.prepareStatement(insertSql);
+            this.preparedStatement.setQueryTimeout(executeTimeoutSec);
+            log.info("PreparedStatement created successfully");
+        } catch (SQLException e) {
+            throw new DatabendConnectorException(
+                    DatabendConnectorErrorCode.SQL_OPERATION_FAILED,
+                    "Failed to prepare statement: " + e.getMessage(),
+                    e);
+        }
+    }
+
+    // This method is no longer needed as raw table creation is handled by
+    // DatabendSinkAggregatedCommitter
+    // private void createRawTable(String database) throws SQLException {

Review Comment:
   please remove useless code.



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