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


##########
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.io.UncheckedIOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.io.CloseableIterable;
+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.util.Pair;
+import org.apache.iceberg.util.ParallelIterable;
+import org.apache.iceberg.util.ThreadPools;
+
+/**
+ * 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 in
+ * parallel using the worker pool. Use {@link 
Builder#executeWith(ExecutorService)} to provide a
+ * custom executor.
+ */
+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 using the table's current snapshot.
+   *
+   * @param table the table to detect file changes for
+   * @return a new Builder
+   */
+  public static Builder builderFor(Table table) {
+    return new Builder(table.currentSnapshot(), table.io(), table.specs());
+  }
+
+  static Builder builderFor(Snapshot snapshot, FileIO io, Map<Integer, 
PartitionSpec> specsById) {
+    return new Builder(snapshot, io, specsById);
+  }
+
+  private ExecutorService workerPool() {
+    return executorService != null ? executorService : 
ThreadPools.getWorkerPool();

Review Comment:
   That's fair, I kind of wanted to default the prod uses to the worker pool, 
but our old behavior (even with the manifest group impl) was single threaded. I 
can just put it as a followup issue if someone wants to benchmark and improve it



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