This is an automated email from the ASF dual-hosted git repository.

corgy pushed a commit to branch dev
in repository https://gitbox.apache.org/repos/asf/seatunnel.git


The following commit(s) were added to refs/heads/dev by this push:
     new 63993a6197 [Fix][Connector-V2] Update waitCompaction value for batch 
mode and writeonly (#9479)
63993a6197 is described below

commit 63993a6197b143ee185e7b060049a9d3ff1fbc4b
Author: xiaochen <[email protected]>
AuthorDate: Thu Jun 26 21:37:27 2025 +0800

    [Fix][Connector-V2] Update waitCompaction value for batch mode and 
writeonly (#9479)
---
 .../seatunnel/paimon/sink/PaimonSinkWriter.java    |  19 +-
 .../paimon/sink/writer/PaimonWriteTest.java        | 295 +++++++++++++++++++++
 2 files changed, 311 insertions(+), 3 deletions(-)

diff --git 
a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java
 
b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java
index bc02cc4bf3..6ba1b877ef 100644
--- 
a/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java
+++ 
b/seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java
@@ -17,6 +17,8 @@
 
 package org.apache.seatunnel.connectors.seatunnel.paimon.sink;
 
+import 
org.apache.seatunnel.shade.com.google.common.annotations.VisibleForTesting;
+
 import org.apache.seatunnel.api.common.JobContext;
 import org.apache.seatunnel.api.configuration.ReadonlyConfig;
 import org.apache.seatunnel.api.sink.SinkWriter;
@@ -28,6 +30,7 @@ import 
org.apache.seatunnel.api.table.schema.event.SchemaChangeEvent;
 import 
org.apache.seatunnel.api.table.schema.handler.TableSchemaChangeEventDispatcher;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
 import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.common.constants.JobMode;
 import org.apache.seatunnel.common.utils.SeaTunnelException;
 import org.apache.seatunnel.connectors.seatunnel.paimon.catalog.PaimonCatalog;
 import 
org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonHadoopConfiguration;
@@ -103,6 +106,8 @@ public class PaimonSinkWriter
     private final TableSchemaChangeEventDispatcher TABLE_SCHEMACHANGER =
             new TableSchemaChangeEventDispatcher();
 
+    private final JobContext jobContext;
+
     public PaimonSinkWriter(
             Context context,
             ReadonlyConfig readonlyConfig,
@@ -113,6 +118,7 @@ public class PaimonSinkWriter
             PaimonHadoopConfiguration paimonHadoopConfiguration) {
         this.sourceTableSchema = catalogTable.getTableSchema();
         this.seaTunnelRowType = this.sourceTableSchema.toPhysicalRowDataType();
+        this.jobContext = jobContext;
         this.paimonTablePath = catalogTable.getTablePath();
         this.paimonCatalog = PaimonCatalog.loadPaimonCatalog(readonlyConfig);
         this.paimonCatalog.open();
@@ -293,9 +299,16 @@ public class PaimonSinkWriter
         }
     }
 
-    private boolean waitCompaction() {
-        CoreOptions.ChangelogProducer changelogProducer =
-                this.paimonFileStoretable.coreOptions().changelogProducer();
+    @VisibleForTesting
+    public boolean waitCompaction() {
+        if (JobMode.BATCH.equals(jobContext.getJobMode())) {
+            return true;
+        }
+        CoreOptions coreOptions = this.paimonFileStoretable.coreOptions();
+        if (coreOptions.writeOnly()) {
+            return false;
+        }
+        CoreOptions.ChangelogProducer changelogProducer = 
coreOptions.changelogProducer();
         return changelogProducer == CoreOptions.ChangelogProducer.LOOKUP
                 || changelogProducer == 
CoreOptions.ChangelogProducer.FULL_COMPACTION;
     }
diff --git 
a/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/writer/PaimonWriteTest.java
 
b/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/writer/PaimonWriteTest.java
new file mode 100644
index 0000000000..f0cb816f31
--- /dev/null
+++ 
b/seatunnel-connectors-v2/connector-paimon/src/test/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/writer/PaimonWriteTest.java
@@ -0,0 +1,295 @@
+/*
+ * 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.paimon.sink.writer;
+
+import org.apache.seatunnel.api.common.JobContext;
+import org.apache.seatunnel.api.common.metrics.MetricsContext;
+import org.apache.seatunnel.api.configuration.ReadonlyConfig;
+import org.apache.seatunnel.api.event.EventListener;
+import org.apache.seatunnel.api.sink.SinkWriter;
+import org.apache.seatunnel.api.table.catalog.CatalogTable;
+import org.apache.seatunnel.api.table.catalog.PhysicalColumn;
+import org.apache.seatunnel.api.table.catalog.TableIdentifier;
+import org.apache.seatunnel.api.table.catalog.TablePath;
+import org.apache.seatunnel.api.table.catalog.TableSchema;
+import org.apache.seatunnel.api.table.type.ArrayType;
+import org.apache.seatunnel.api.table.type.BasicType;
+import org.apache.seatunnel.api.table.type.DecimalType;
+import org.apache.seatunnel.api.table.type.LocalTimeType;
+import org.apache.seatunnel.api.table.type.MapType;
+import org.apache.seatunnel.common.constants.JobMode;
+import org.apache.seatunnel.connectors.seatunnel.paimon.catalog.PaimonCatalog;
+import 
org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonHadoopConfiguration;
+import 
org.apache.seatunnel.connectors.seatunnel.paimon.config.PaimonSinkConfig;
+import org.apache.seatunnel.connectors.seatunnel.paimon.sink.PaimonSinkWriter;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+public class PaimonWriteTest {
+
+    private PaimonCatalog paimonCatalog;
+    private TableSchema.Builder schemaBuilder;
+    private final String CATALOG_NAME = "paimon_catalog";
+    private final String DATABASE_NAME = "test_default";
+    private final String TABLE_NAME = "test_table";
+    private PaimonSinkWriter paimonSinkWriter;
+    private ReadonlyConfig readonlyConfig;
+    private SinkWriter.Context context;
+
+    @BeforeEach
+    public void before() {
+
+        Map<String, Object> properties = new HashMap<>();
+        properties.put("warehouse", "/tmp/paimon");
+        properties.put("plugin_name", "Paimon");
+        properties.put("database", DATABASE_NAME);
+        properties.put("table", TABLE_NAME);
+        Map<String, String> writeProps = new HashMap<>();
+        writeProps.put("write-only", "true");
+        properties.put("paimon.table.write-props", writeProps);
+        readonlyConfig = ReadonlyConfig.fromMap(properties);
+        paimonCatalog = new PaimonCatalog(CATALOG_NAME, readonlyConfig);
+        paimonCatalog.open();
+        paimonCatalog.createDatabase(TablePath.of(DATABASE_NAME, TABLE_NAME), 
false);
+        this.schemaBuilder =
+                TableSchema.builder()
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_map",
+                                        new MapType<>(BasicType.STRING_TYPE, 
BasicType.STRING_TYPE),
+                                        (Long) null,
+                                        true,
+                                        null,
+                                        null))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_array",
+                                        ArrayType.STRING_ARRAY_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_array"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_string",
+                                        BasicType.STRING_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_string"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_boolean",
+                                        BasicType.BOOLEAN_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_boolean"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_tinyint",
+                                        BasicType.INT_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_tinyint"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_smallint",
+                                        BasicType.INT_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_smallint"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_int",
+                                        BasicType.INT_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_int"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_bigint",
+                                        BasicType.LONG_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_bigint"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_float",
+                                        BasicType.FLOAT_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_float"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_double",
+                                        BasicType.DOUBLE_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_double"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_decimal",
+                                        new DecimalType(10, 2),
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_decimal"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_bytes",
+                                        BasicType.BYTE_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_bytes"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_date",
+                                        LocalTimeType.LOCAL_DATE_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_date"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_timestamp",
+                                        LocalTimeType.LOCAL_DATE_TIME_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_timestamp"))
+                        .column(
+                                PhysicalColumn.of(
+                                        "c_time",
+                                        LocalTimeType.LOCAL_TIME_TYPE,
+                                        (Long) null,
+                                        false,
+                                        null,
+                                        "c_time"));
+        paimonCatalog.createTable(
+                TablePath.of(DATABASE_NAME, TABLE_NAME),
+                CatalogTable.of(
+                        TableIdentifier.of(CATALOG_NAME, DATABASE_NAME, 
TABLE_NAME),
+                        schemaBuilder.build(),
+                        new HashMap<>(),
+                        new ArrayList<>(),
+                        "test table"),
+                false);
+
+        context =
+                new SinkWriter.Context() {
+                    @Override
+                    public int getIndexOfSubtask() {
+                        return 0;
+                    }
+
+                    @Override
+                    public MetricsContext getMetricsContext() {
+                        return null;
+                    }
+
+                    @Override
+                    public EventListener getEventListener() {
+                        return null;
+                    }
+                };
+    }
+
+    @Test
+    void testWaitCompaction() throws Exception {
+
+        JobContext jobContext = new JobContext();
+        jobContext.setJobMode(JobMode.STREAMING);
+        TablePath tablePath = TablePath.of(DATABASE_NAME, TABLE_NAME);
+        paimonSinkWriter =
+                new PaimonSinkWriter(
+                        context,
+                        readonlyConfig,
+                        paimonCatalog.getTable(tablePath),
+                        paimonCatalog.getPaimonTable(tablePath),
+                        jobContext,
+                        new PaimonSinkConfig(readonlyConfig),
+                        new PaimonHadoopConfiguration());
+        Assertions.assertFalse(paimonSinkWriter.waitCompaction());
+
+        jobContext.setJobMode(JobMode.BATCH);
+        paimonSinkWriter =
+                new PaimonSinkWriter(
+                        context,
+                        readonlyConfig,
+                        paimonCatalog.getTable(tablePath),
+                        paimonCatalog.getPaimonTable(tablePath),
+                        jobContext,
+                        new PaimonSinkConfig(readonlyConfig),
+                        new PaimonHadoopConfiguration());
+        Assertions.assertTrue(paimonSinkWriter.waitCompaction());
+
+        Map<String, Object> properties = new HashMap<>();
+        properties.put("warehouse", "/tmp/paimon");
+        properties.put("plugin_name", "Paimon");
+        properties.put("database", DATABASE_NAME);
+        properties.put("table", TABLE_NAME);
+        Map<String, String> writeProps = new HashMap<>();
+        writeProps.put("changelog-producer", "lookup");
+        properties.put("paimon.table.write-props", writeProps);
+        readonlyConfig = ReadonlyConfig.fromMap(properties);
+        paimonSinkWriter =
+                new PaimonSinkWriter(
+                        context,
+                        readonlyConfig,
+                        paimonCatalog.getTable(tablePath),
+                        paimonCatalog.getPaimonTable(tablePath),
+                        jobContext,
+                        new PaimonSinkConfig(readonlyConfig),
+                        new PaimonHadoopConfiguration());
+        Assertions.assertTrue(paimonSinkWriter.waitCompaction());
+
+        writeProps.put("changelog-producer", "full-compaction");
+        readonlyConfig = ReadonlyConfig.fromMap(properties);
+        paimonSinkWriter =
+                new PaimonSinkWriter(
+                        context,
+                        readonlyConfig,
+                        paimonCatalog.getTable(tablePath),
+                        paimonCatalog.getPaimonTable(tablePath),
+                        jobContext,
+                        new PaimonSinkConfig(readonlyConfig),
+                        new PaimonHadoopConfiguration());
+        Assertions.assertTrue(paimonSinkWriter.waitCompaction());
+    }
+
+    @AfterEach
+    public void after() {
+        paimonCatalog.dropDatabase(TablePath.of(DATABASE_NAME, TABLE_NAME), 
false);
+        paimonCatalog.close();
+    }
+}

Reply via email to