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


##########
core/src/main/java/org/apache/iceberg/SnapshotChanges.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.util.Tasks;
+
+/**
+ * Helper class for retrieving file changes in a snapshot with caching.
+ *
+ * <p>This class caches the results of file change detection operations, 
making it efficient to
+ * query multiple file change types for the same snapshot. By default, 
manifests are read
+ * sequentially. Use {@link Builder#executeWith(ExecutorService)} to enable 
parallel reading.
+ */
+public class SnapshotChanges {
+  private final Snapshot snapshot;
+  private final FileIO io;
+  private final Map<Integer, PartitionSpec> specsById;
+  private final ExecutorService executorService;
+
+  private List<DataFile> addedDataFiles = null;
+  private List<DataFile> removedDataFiles = null;
+  private List<DeleteFile> addedDeleteFiles = null;
+  private List<DeleteFile> removedDeleteFiles = null;
+
+  private SnapshotChanges(
+      Snapshot snapshot,
+      FileIO io,
+      Map<Integer, PartitionSpec> specsById,
+      ExecutorService executorService) {
+    Preconditions.checkArgument(snapshot != null, "Snapshot cannot be null");
+    Preconditions.checkArgument(io != null, "FileIO cannot be null");
+    Preconditions.checkArgument(specsById != null, "Partition specs cannot be 
null");
+    this.snapshot = snapshot;
+    this.io = io;
+    this.specsById = specsById;
+    this.executorService = executorService;
+  }
+
+  /**
+   * Create a builder for SnapshotChanges.
+   *
+   * @param snapshot the snapshot to detect file changes for
+   * @param io a {@link FileIO} instance used for reading files from storage
+   * @param specsById a map of partition spec IDs to partition specs
+   * @return a new Builder
+   */
+  public static Builder builder(
+      Snapshot snapshot, FileIO io, Map<Integer, PartitionSpec> specsById) {
+    return new Builder(snapshot, io, specsById);
+  }
+
+  /** Returns all data files added to the table in this snapshot */
+  public Iterable<DataFile> addedDataFiles() {
+    if (addedDataFiles == null) {
+      cacheDataFileChanges();
+    }
+    return addedDataFiles;
+  }
+
+  /** Returns all data files removed from the table in this snapshot. */
+  public Iterable<DataFile> removedDataFiles() {
+    if (removedDataFiles == null) {
+      cacheDataFileChanges();
+    }
+    return removedDataFiles;
+  }
+
+  /** Returns all delete files added to the table in this snapshot. */
+  public Iterable<DeleteFile> addedDeleteFiles() {
+    if (addedDeleteFiles == null) {
+      cacheDeleteFileChanges();
+    }
+    return addedDeleteFiles;
+  }
+
+  /** Returns all delete files removed from the table in this snapshot. */
+  public Iterable<DeleteFile> removedDeleteFiles() {
+    if (removedDeleteFiles == null) {
+      cacheDeleteFileChanges();
+    }
+    return removedDeleteFiles;
+  }
+
+  private void cacheDataFileChanges() {
+    List<ManifestFile> changedManifests =
+        Lists.newArrayList(
+            Iterables.filter(
+                Iterables.filter(
+                    snapshot.allManifests(io),
+                    manifest -> manifest.content() == ManifestContent.DATA),
+                manifest -> Objects.equals(manifest.snapshotId(), 
snapshot.snapshotId())));
+
+    Queue<DataFileChanges> fileChangesByManifest = 
Queues.newConcurrentLinkedQueue();
+    Tasks.foreach(changedManifests)
+        .stopOnFailure()
+        .throwFailureWhenFinished()
+        .executeWith(executorService)
+        .run(manifest -> 
fileChangesByManifest.add(readDataFileChanges(manifest)));

Review Comment:
   Oh, I see. The delete code used `readDeleteManifest` directly, so this is 
probably just being consistent.
   
   I think I'd still recommend using `ParallelIterable`. Here's what I came up 
with:
   
   ```java
     private Iterable<Pair<ManifestEntry.Status, DeleteFile>> 
readDeleteFiles(ManifestFile manifest) {
       Iterable<ManifestEntry<DeleteFile>> entries = 
ManifestFiles.readDeleteManifest(manifest, fileIO, null).entries();
       Iterable<Pair<ManifestEntry.Status, DeleteFile>> copied = 
Iterables.transform(entries, entry ->
           switch (entry.status()) {
             case ADDED -> Pair.of(ManifestEntry.Status.ADDED, 
entry.file().copy());
             case DELETED -> Pair.of(ManifestEntry.Status.DELETED, 
entry.file().copyWithoutStats());
             default -> null;
           });
       return Iterables.filter(copied, java.util.Objects::nonNull);
     }
   
     private void cacheDeleteFileChanges() {
       ImmutableList.Builder<DeleteFile> adds = ImmutableList.builder();
       ImmutableList.Builder<DeleteFile> deletes = ImmutableList.builder();
   
       Iterable<ManifestFile> changedManifests =
           Iterables.filter(
               deleteManifests(fileIO), manifest -> 
Objects.equal(manifest.snapshotId(), snapshotId));
   
       Iterable<Iterable<Pair<ManifestEntry.Status, DeleteFile>>> 
changedDeletes = Iterables.transform(changedManifests, this::readDeleteFiles);
   
       try (CloseableIterable<Pair<ManifestEntry.Status, DeleteFile>> pairs = 
new ParallelIterable<>(changedDeletes, ThreadPools.getWorkerPool())) {
         for (Pair<ManifestEntry.Status, DeleteFile> delete : pairs) {
           switch (delete.first()) {
             case ADDED -> adds.add(delete.second());
             case DELETED -> deletes.add(delete.second());
           }
         }
       } catch (IOException e) {
         throw new UncheckedIOException("Failed to close manifest reader", e);
       }
   
       this.addedDeleteFiles = adds.build();
       this.removedDeleteFiles = deletes.build();
     }
   ```
   
   This uses `Pair` because the files need to be copied before returning them 
to `ParallelIterable`, but that would lose the added/deleted status. This is 
the kind of thing that inspired inverting the manifest_entry / delete_file 
relationship in v4 so that delete_file contains tracking_info.



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