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


##########
core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java:
##########
@@ -1073,6 +1088,125 @@ private List<ManifestFile> newDeleteFilesAsManifests() {
     return cachedNewDeleteManifests;
   }
 
+  // Merge duplicates, internally takes care of updating newDeleteFilesBySpec 
to remove
+  // duplicates and add the newly merged DV
+  private void mergeDVsAndWrite() {
+    Map<String, DeleteFileSet> dataFilesWithDuplicateDVs =
+        dvsByReferencedFile.entrySet().stream()
+            .filter(entry -> entry.getValue().size() > 1)
+            .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    List<MergedDVContent> mergedDVs = 
Collections.synchronizedList(Lists.newArrayList());
+    Tasks.foreach(dataFilesWithDuplicateDVs.entrySet())
+        .executeWith(ThreadPools.getDeleteWorkerPool())
+        .stopOnFailure()
+        .throwFailureWhenFinished()
+        .run(
+            entry -> {
+              String referencedLocation = entry.getKey();
+              DeleteFileSet duplicateDVs = entry.getValue();
+              mergedDVs.add(mergePositions(referencedLocation, duplicateDVs));
+            });
+
+    // Update newDeleteFilesBySpec to remove all the duplicates
+    mergedDVs.forEach(
+        mergedDV -> 
newDeleteFilesBySpec.get(mergedDV.specId).removeAll(mergedDV.duplicateDVs));
+
+    writeMergedDVs(mergedDVs);
+  }
+
+  // Produces a Puffin per partition spec containing the merged DVs for that 
spec
+  private void writeMergedDVs(List<MergedDVContent> mergedDVs) {
+    try (DVFileWriter dvFileWriter =
+        new BaseDVFileWriter(
+            // Use an unpartitioned spec for the location provider for the 
puffin containing
+            // all the merged DVs
+            OutputFileFactory.builderFor(
+                    ops(), PartitionSpec.unpartitioned(), FileFormat.PUFFIN, 
1, 1)
+                .build(),
+            path -> null)) {
+
+      for (MergedDVContent mergedDV : mergedDVs) {
+        LOG.warn(
+            "Merged {} duplicate deletion vectors for data file {} in table 
{}. The duplicate DVs are orphaned, and writers should merge DVs per file 
before committing",
+            mergedDV.duplicateDVs.size(),
+            mergedDV.referencedLocation,
+            tableName);
+        dvFileWriter.delete(
+            mergedDV.referencedLocation,
+            mergedDV.mergedPositions,
+            spec(mergedDV.specId),
+            mergedDV.partition);
+      }
+
+      dvFileWriter.close();
+      DeleteWriteResult result = dvFileWriter.result();
+      result.deleteFiles().forEach(this::addPendingDelete);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  private void addPendingDelete(DeleteFile file) {
+    newDeleteFilesBySpec
+        .get(file.specId())
+        .add(Delegates.pendingDeleteFile(file, file.dataSequenceNumber()));
+  }
+
+  // Data class for referenced file, the duplicate DVs, the merged position 
delete index,
+  // partition spec and tuple
+  private static class MergedDVContent {
+    private final DeleteFileSet duplicateDVs;
+    private final String referencedLocation;
+    private final PositionDeleteIndex mergedPositions;
+    private final int specId;
+    private final StructLike partition;
+
+    MergedDVContent(
+        String referencedLocation,
+        DeleteFileSet duplicateDVs,
+        PositionDeleteIndex mergedPositions,
+        int specId,
+        StructLike partition) {
+      this.referencedLocation = referencedLocation;
+      this.duplicateDVs = duplicateDVs;
+      this.mergedPositions = mergedPositions;
+      this.specId = specId;
+      this.partition = partition;
+    }
+  }
+
+  // Merges the position indices for the duplicate DVs for a given referenced 
file
+  private MergedDVContent mergePositions(String referencedLocation, 
DeleteFileSet duplicateDVs) {
+    Iterator<DeleteFile> dvIterator = duplicateDVs.iterator();
+    DeleteFile firstDV = dvIterator.next();
+    PositionDeleteIndex mergedPositions = Deletes.readDV(firstDV, ops().io(), 
ops().encryption());
+    while (dvIterator.hasNext()) {
+      DeleteFile dv = dvIterator.next();
+      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()),

Review Comment:
   I don't think there is a guarantee about the underlying type of `partition` 
(it should be `StructLike`, I think). If that's the case, then `Objects.equals` 
is not going to work all of the time. If you want to compare partitions then 
you'd need a struct comparator for the struct produced by the partition spec. 
Honestly, I think that's a bit overkill if you know that the referenced data 
file matched though.



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