Copilot commented on code in PR #9595:
URL: https://github.com/apache/seatunnel/pull/9595#discussion_r2218124481


##########
seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java:
##########
@@ -122,31 +130,39 @@ public PaimonSinkWriter(
         this.paimonTablePath = catalogTable.getTablePath();
         this.paimonCatalog = PaimonCatalog.loadPaimonCatalog(readonlyConfig);
         this.paimonCatalog.open();
-        this.paimonFileStoretable = (FileStoreTable) paimonFileStoretable;
+        this.paimonFileStoretable = (FileStoreTable) paimonTable;
         CoreOptions.ChangelogProducer changelogProducer =
                 this.paimonFileStoretable.coreOptions().changelogProducer();
         if (Objects.nonNull(paimonSinkConfig.getChangelogProducer())
                 && changelogProducer != 
paimonSinkConfig.getChangelogProducer()) {
             log.warn(
                     "configured the props named 'changelog-producer' which is 
not compatible with the options in table , so it will use the table's 
'changelog-producer'");
         }
+        this.rowAssignerChannelComputer =
+                new RowAssignerChannelComputer(
+                        paimonFileStoretable.schema(), 
context.getNumberOfParallelSubtasks());
+        
rowAssignerChannelComputer.setup(context.getNumberOfParallelSubtasks());
+        this.parallelism = context.getNumberOfParallelSubtasks();
         this.paimonSinkConfig = paimonSinkConfig;
         this.sinkPaimonTableSchema = this.paimonFileStoretable.schema();
         this.newTableWrite();
         BucketMode bucketMode = this.paimonFileStoretable.bucketMode();
-        this.dynamicBucket =
-                BucketMode.DYNAMIC == bucketMode || BucketMode.GLOBAL_DYNAMIC 
== bucketMode;
-        int bucket = ((FileStoreTable) 
paimonFileStoretable).coreOptions().bucket();
+        // 
https://paimon.apache.org/docs/master/primary-key-table/data-distribution/#dynamic-bucket
+        // When you need cross partition upsert (primary keys not contain all 
partition fields),
+        // Dynamic Bucket mode directly maintains the mapping of keys to 
partition and bucket, uses
+        // local disks, and initializes indexes by reading all existing keys 
in the table when
+        // starting job. For tables with a large amount of data, there will be 
a significant loss in
+        // performance. Moreover, initialization takes a long time. This mode 
is not supported at
+        // this time.
+        if (BucketMode.GLOBAL_DYNAMIC == bucketMode) {
+            throw new UnsupportedOperationException(
+                    "Cross Partitions Upsert Dynamic Bucket Mode is not 
sopported.");

Review Comment:
   There is a spelling error in the error message. 'sopported' should be 
'supported'.
   ```suggestion
                       "Cross Partitions Upsert Dynamic Bucket Mode is not 
supported.");
   ```



##########
seatunnel-e2e/seatunnel-connector-v2-e2e/connector-paimon-e2e/src/test/java/org/apache/seatunnel/e2e/connector/paimon/PaimonSinkDynamicBucketIT.java:
##########
@@ -144,6 +213,95 @@ private void deleteTable() {
         }
     }
 
+    @TestTemplate
+    public void testWriteForDifferentParallelism(TestContainer container)
+            throws IOException, InterruptedException, SQLException {
+        // parallelism = 3
+        Container.ExecResult textWriteResult1 =
+                
container.executeJob("/mysql_jdbc_to_dynamic_bucket_paimon_case1.conf");
+        Assertions.assertEquals(0, textWriteResult1.getExitCode());
+        try (Connection jdbcConnection = bucketDatabase.getJdbcConnection();
+                Statement statement = jdbcConnection.createStatement()) {
+            statement.executeUpdate(
+                    "update bucket.test_dynamic_bucket set version = '2' where 
id <= 102");
+            statement.executeUpdate(
+                    "update bucket.test_dynamic_bucket set version = '3' where 
id = 105");
+            statement.executeUpdate(
+                    "update bucket.test_dynamic_bucket set version = '4' where 
id = 109");
+        }
+        // parallelism = 1
+        Container.ExecResult textWriteResult2 =
+                
container.executeJob("/mysql_jdbc_to_dynamic_bucket_paimon_case2.conf");
+        Assertions.assertEquals(0, textWriteResult2.getExitCode());
+        List<String> parallelism_1 = vertifyData(container);
+
+        // parallelism = 2
+        Container.ExecResult textWriteResult3 =
+                
container.executeJob("/mysql_jdbc_to_dynamic_bucket_paimon_case3.conf");
+        Assertions.assertEquals(0, textWriteResult3.getExitCode());
+
+        List<String> parallelism_2 = vertifyData(container);
+        Assertions.assertEquals(parallelism_1, parallelism_2);
+    }
+
+    private List<String> vertifyData(TestContainer container) {

Review Comment:
   Method name has a spelling error. 'vertifyData' should be 'verifyData'.
   ```suggestion
           List<String> parallelism_1 = verifyData(container);
   
           // parallelism = 2
           Container.ExecResult textWriteResult3 =
                   
container.executeJob("/mysql_jdbc_to_dynamic_bucket_paimon_case3.conf");
           Assertions.assertEquals(0, textWriteResult3.getExitCode());
   
           List<String> parallelism_2 = verifyData(container);
           Assertions.assertEquals(parallelism_1, parallelism_2);
       }
   
       private List<String> verifyData(TestContainer container) {
   ```



##########
seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/PaimonSinkWriter.java:
##########
@@ -199,8 +215,22 @@ public void write(SeaTunnelRow element) throws IOException 
{
             PaimonSecurityContext.runSecured(
                     () -> {
                         if (dynamicBucket) {
-                            int bucket = bucketAssigner.assign(rowData);
-                            tableWrite.write(rowData, bucket);
+                            // The result of calculating the remainder of the 
parallelism using the
+                            // hash code of the primary key must be consistent 
with the task
+                            // sequence number.
+                            PaimonBucketAssigner bucketAssigner =
+                                    
PaimonBucketAssignerFactory.getBucketAssigner(
+                                            paimonTablePath,
+                                            paimonFileStoretable,
+                                            parallelism,
+                                            
rowAssignerChannelComputer.channel(rowData));
+                            // When multiple threads call assigner.assign() 
simultaneously, they can
+                            // corrupt the internal hash map structure, 
leading to the
+                            // ArrayIndexOutOfBoundsException during rehashing 
operations
+                            synchronized (bucketAssigner) {

Review Comment:
   Synchronizing on the bucketAssigner instance may cause performance 
bottlenecks. Consider using a more granular locking mechanism or concurrent 
data structures to reduce contention.



##########
seatunnel-connectors-v2/connector-paimon/src/main/java/org/apache/seatunnel/connectors/seatunnel/paimon/sink/bucket/PaimonBucketAssignerFactory.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.bucket;
+
+import org.apache.seatunnel.api.table.catalog.TablePath;
+
+import org.apache.paimon.table.Table;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class PaimonBucketAssignerFactory {
+    private static final ConcurrentHashMap<TablePath, Map<Integer, 
PaimonBucketAssigner>>

Review Comment:
   The static ConcurrentHashMap may grow unbounded and cause memory leaks if 
tables are frequently created/dropped. Consider implementing a cleanup 
mechanism or using WeakReferences.



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