rdblue commented on code in PR #15006:
URL: https://github.com/apache/iceberg/pull/15006#discussion_r2850178568


##########
core/src/main/java/org/apache/iceberg/DVUtil.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Supplier;
+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.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.Tasks;
+
+class DVUtil {
+  private DVUtil() {}
+
+  /**
+   * Merges duplicate DVs for the same data file and writes the merged DV 
Puffin files. If there is
+   * exactly 1 DV for a given data file then it is return as is
+   *
+   * @param dvsByFile map of data file location to DVs
+   * @return a list containing both any newly merged DVs and any DVs that are 
already valid
+   */
+  static List<DeleteFile> mergeAndWriteDVsIfRequired(
+      Map<String, List<DeleteFile>> dvsByFile,
+      Supplier<OutputFile> dvOutputFile,
+      FileIO fileIO,
+      Map<Integer, PartitionSpec> specs,
+      ExecutorService pool) {
+    Map<String, List<DeleteFile>> duplicateDVsByFile =

Review Comment:
   There are a couple of things that make this method fairly complicated. 
First, using a map of lists ends up causing this to do a fair amount of stream 
manipulation. Using a Guava multimap cleans that up quite a bit.
   
   Second, this doesn't need to process the input map multiple times. This is 
cleaner if you process it once, keep the DVs that don't need to be merged in an 
output list, and also accumulate the partition and sequence number info for 
later at the same time. Doing that also simplifies the write and validate 
methods because you can identify the expected data sequence number and 
partition information here rather than complicating the loops in later methods.
   
   Here's what I came up with:
   
   ```java
     static List<DeleteFile> mergeAndWriteDVsIfRequired(
         Map<String, List<DeleteFile>> dvsByFile,
         Supplier<OutputFile> dvOutputFile,
         FileIO fileIO,
         Map<Integer, PartitionSpec> specs,
         ExecutorService pool) {
   
       List<DeleteFile> finalDVs = Lists.newArrayList();
       Multimap<String, DeleteFile> duplicates =
           Multimaps.newListMultimap(Maps.newHashMap(), Lists::newArrayList);
       Map<String, Pair<PartitionSpec, StructLike>> partitions = 
Maps.newHashMap();
   
       for (Map.Entry<String, List<DeleteFile>> entry : dvsByFile.entrySet()) {
         if (entry.getValue().size() > 1) {
           duplicates.putAll(entry.getKey(), entry.getValue());
           DeleteFile first = entry.getValue().get(0);
           partitions.put(entry.getKey(), Pair.of(specs.get(first.specId()), 
first.partition()));
         } else {
           finalDVs.addAll(entry.getValue());
         }
       }
   
       if (duplicates.isEmpty()) {
         return finalDVs;
       }
   
       validateCanMerge(duplicates, partitions);
   
       Map<String, PositionDeleteIndex> deletes =
           readAndMergeDVs(duplicates.values().toArray(DeleteFile[]::new), 
fileIO, pool);
   
       finalDVs.addAll(writeDVs(deletes, partitions, dvOutputFile));
   
       return finalDVs;
     }
   ```



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