Copilot commented on code in PR #6497:
URL: https://github.com/apache/paimon/pull/6497#discussion_r2480450040


##########
paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/format/PaimonFormatTable.scala:
##########
@@ -101,37 +104,24 @@ private case class FormatTableBatchWrite(
     !(overwriteDynamic && overwritePartitions.exists(_.nonEmpty)),
     "Cannot overwrite dynamically and by filter both")
 
+  private val batchWriteBuilder = {
+    val builder = table.newBatchWriteBuilder()
+    overwritePartitions.foreach(partitions => 
builder.withOverwrite(partitions.asJava))

Review Comment:
   The `batchWriteBuilder` should handle the `overwriteDynamic` case. When 
`overwriteDynamic` is true and `overwritePartitions` is `None` or empty, 
`withOverwrite` should be called with an empty map to indicate a full table 
overwrite. Currently, if `overwriteDynamic` is true but `overwritePartitions` 
is `None`, no overwrite flag is set on the builder.
   ```suggestion
       overwritePartitions.foreach(partitions => 
builder.withOverwrite(partitions.asJava))
       if (overwriteDynamic && overwritePartitions.forall(_.isEmpty)) {
         builder.withOverwrite(Map.empty[String, String].asJava)
       }
   ```



##########
paimon-core/src/main/java/org/apache/paimon/table/format/FormatTableCommit.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.paimon.table.format;
+
+import org.apache.paimon.fs.FileIO;
+import org.apache.paimon.fs.FileStatus;
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.fs.TwoPhaseOutputStream;
+import org.apache.paimon.metrics.MetricRegistry;
+import org.apache.paimon.stats.Statistics;
+import org.apache.paimon.table.sink.BatchTableCommit;
+import org.apache.paimon.table.sink.CommitMessage;
+import org.apache.paimon.table.sink.TableCommit;
+
+import javax.annotation.Nullable;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.StringJoiner;
+
+import static 
org.apache.paimon.table.format.FormatBatchWriteBuilder.validateStaticPartition;
+
+/** Commit for Format Table. */
+public class FormatTableCommit implements BatchTableCommit {
+
+    private String location;
+    private final boolean formatTablePartitionOnlyValueInPath;
+    private FileIO fileIO;
+    private List<String> partitionKeys;
+    protected Map<String, String> staticPartitions;
+    protected boolean overwrite = false;
+
+    public FormatTableCommit(
+            String location,
+            List<String> partitionKeys,
+            FileIO fileIO,
+            boolean formatTablePartitionOnlyValueInPath,
+            boolean overwrite,
+            @Nullable Map<String, String> staticPartitions) {
+        this.location = location;
+        this.fileIO = fileIO;
+        this.formatTablePartitionOnlyValueInPath = 
formatTablePartitionOnlyValueInPath;
+        validateStaticPartition(staticPartitions, partitionKeys);
+        this.staticPartitions = staticPartitions;
+        this.overwrite = overwrite;
+        this.partitionKeys = partitionKeys;
+    }
+
+    @Override
+    public void commit(List<CommitMessage> commitMessages) {
+        try {
+            List<TwoPhaseOutputStream.Committer> committers = new 
ArrayList<>();
+            for (CommitMessage commitMessage : commitMessages) {
+                if (commitMessage instanceof TwoPhaseCommitMessage) {
+                    committers.add(((TwoPhaseCommitMessage) 
commitMessage).getCommitter());
+                } else {
+                    throw new RuntimeException(
+                            "Unsupported commit message type: "
+                                    + commitMessage.getClass().getName());
+                }
+            }
+            if (overwrite && staticPartitions != null && 
!staticPartitions.isEmpty()) {
+                Path partitionPath =
+                        new Path(
+                                location,
+                                buildPartitionName(
+                                        staticPartitions,
+                                        formatTablePartitionOnlyValueInPath,
+                                        partitionKeys));
+                deletePreviousDataFile(partitionPath);
+            } else if (overwrite) {
+                Set<Path> partitionPaths = new HashSet<>();
+                for (TwoPhaseOutputStream.Committer c : committers) {
+                    partitionPaths.add(c.targetFilePath().getParent());
+                }
+                for (Path p : partitionPaths) {
+                    deletePreviousDataFile(p);
+                }
+            }
+            for (TwoPhaseOutputStream.Committer committer : committers) {
+                committer.commit(this.fileIO);
+            }
+        } catch (Exception e) {
+            this.abort(commitMessages);
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static String buildPartitionName(
+            Map<String, String> partitionSpec,
+            boolean formatTablePartitionOnlyValueInPath,
+            List<String> partitionKeys) {
+        if (partitionSpec.isEmpty() || partitionKeys.isEmpty()) {
+            return "";
+        }
+        StringJoiner joiner = new StringJoiner("/");
+        for (int i = 0; i < partitionSpec.size(); i++) {

Review Comment:
   The loop iterates based on `partitionSpec.size()` but accesses 
`partitionKeys.get(i)`, which could cause an IndexOutOfBoundsException if 
`partitionSpec.size() > partitionKeys.size()`. The loop should iterate over 
`partitionKeys` in order, checking if each key exists in `partitionSpec`.
   ```suggestion
           for (int i = 0; i < partitionKeys.size(); i++) {
   ```



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