amogh-jahagirdar commented on code in PR #15006:
URL: https://github.com/apache/iceberg/pull/15006#discussion_r2730034212


##########
core/src/main/java/org/apache/iceberg/DVUtil.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
+import org.apache.iceberg.deletes.BaseDVFileWriter;
+import org.apache.iceberg.deletes.DVFileWriter;
+import org.apache.iceberg.deletes.Deletes;
+import org.apache.iceberg.deletes.PositionDeleteIndex;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class DVUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(DVUtil.class);
+
+  private DVUtil() {}
+
+  /**
+   * Merges duplicate DVs for the same data file and writes the merged DV 
Puffin files.
+   *
+   * @param duplicateDVsByReferencedFile map of data file location to 
duplicate DVs (all entries
+   *     must have size > 1)
+   * @return newly merged DVs
+   */
+  static List<DeleteFile> mergeDVsAndWrite(
+      TableOperations ops,
+      Map<String, List<DeleteFile>> duplicateDVsByReferencedFile,
+      String tableName,
+      ExecutorService threadpool) {
+    Map<String, PositionDeleteIndex> mergedIndices =
+        duplicateDVsByReferencedFile.entrySet().stream()
+            .collect(
+                Collectors.toMap(
+                    Map.Entry::getKey, entry -> mergePositions(ops, 
entry.getValue(), threadpool)));
+
+    return writeMergedDVs(
+        mergedIndices, duplicateDVsByReferencedFile, ops, tableName, 
ops.current().specsById());
+  }
+
+  // Merges the position indices for the duplicate DVs for a given referenced 
file
+  private static PositionDeleteIndex mergePositions(
+      TableOperations ops, List<DeleteFile> dvsForFile, ExecutorService pool) {
+    Preconditions.checkArgument(dvsForFile.size() > 1, "Expected more than 1 
DV");
+    PositionDeleteIndex[] duplicateDVIndices = new 
PositionDeleteIndex[dvsForFile.size()];
+    Tasks.range(dvsForFile.size())
+        .executeWith(pool)
+        .stopOnFailure()
+        .throwFailureWhenFinished()
+        .run(
+            i -> {
+              duplicateDVIndices[i] = Deletes.readDV(dvsForFile.get(i), 
ops.io(), ops.encryption());
+            });
+    PositionDeleteIndex mergedPositions = duplicateDVIndices[0];
+    DeleteFile firstDV = dvsForFile.get(0);
+    for (int i = 1; i < duplicateDVIndices.length; i++) {
+      DeleteFile dv = dvsForFile.get(i);
+      Preconditions.checkArgument(
+          Objects.equals(dv.dataSequenceNumber(), 
firstDV.dataSequenceNumber()),
+          "Cannot merge duplicate added DVs when data sequence numbers are 
different, "
+              + "expected all to be added with sequence %s, but got %s",
+          firstDV.dataSequenceNumber(),
+          dv.dataSequenceNumber());
+
+      Preconditions.checkArgument(
+          dv.specId() == firstDV.specId(),
+          "Cannot merge duplicate added DVs when partition specs are 
different, "
+              + "expected all to be added with spec %s, but got %s",
+          firstDV.specId(),
+          dv.specId());
+
+      Preconditions.checkArgument(
+          Objects.equals(dv.partition(), firstDV.partition()),
+          "Cannot merge duplicate added DVs when partition tuples are 
different");

Review Comment:
   @rdblue let me know if you feel strongly about this check. While it is 
`StructLike` and doesn't guarantee an equals implementation, the way I look at 
it is the following:
   
   1. Generally it'll be PartitionData which does do a type and value by value 
comparison.
   2. Even if it ends up being a StructLike which doesn't override equals, then 
it's a reference equality which will worst case be a false positive and just 
fail the commit.
   
   Another rationale behind these checks is that if a writer produces duplicate 
DVs, there's also a chance of some kind of metadata record reuse issue from the 
writer and this felt like an easy sanity check.
   
   Alternatively, we could just simplify this and remove these validations by 
assuming that the duplicate DVs are OK by every other dimension.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to