wypoon commented on code in PR #10935:
URL: https://github.com/apache/iceberg/pull/10935#discussion_r1757391061


##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -63,33 +60,43 @@ protected CloseableIterable<ChangelogScanTask> doPlanFiles(
       return CloseableIterable.empty();
     }
 
-    Set<Long> changelogSnapshotIds = toSnapshotIds(changelogSnapshots);
+    Map<Long, Integer> snapshotOrdinals = 
computeSnapshotOrdinals(changelogSnapshots);
 
-    Set<ManifestFile> newDataManifests =
-        FluentIterable.from(changelogSnapshots)
-            .transformAndConcat(snapshot -> 
snapshot.dataManifests(table().io()))
-            .filter(manifest -> 
changelogSnapshotIds.contains(manifest.snapshotId()))
-            .toSet();
-
-    ManifestGroup manifestGroup =
-        new ManifestGroup(table().io(), newDataManifests, ImmutableList.of())
-            .specsById(table().specs())
-            .caseSensitive(isCaseSensitive())
-            .select(scanColumns())
-            .filterData(filter())
-            .filterManifestEntries(entry -> 
changelogSnapshotIds.contains(entry.snapshotId()))
-            .ignoreExisting()
-            .columnsToKeepStats(columnsToKeepStats());
-
-    if (shouldIgnoreResiduals()) {
-      manifestGroup = manifestGroup.ignoreResiduals();
-    }
-
-    if (newDataManifests.size() > 1 && shouldPlanWithExecutor()) {
-      manifestGroup = manifestGroup.planWith(planExecutor());
-    }
+    // map of delete file to the snapshot where the delete file is added
+    // the delete file is keyed by its path, and the snapshot is represented 
by the snapshot ordinal
+    Map<String, Integer> deleteFileToSnapshotOrdinal =
+        computeDeleteFileToSnapshotOrdinal(changelogSnapshots, 
snapshotOrdinals);
 
-    return manifestGroup.plan(new 
CreateDataFileChangeTasks(changelogSnapshots));
+    Iterable<CloseableIterable<ChangelogScanTask>> plans =
+        FluentIterable.from(changelogSnapshots)
+            .transform(
+                snapshot -> {
+                  List<ManifestFile> dataManifests = 
snapshot.dataManifests(table().io());
+                  List<ManifestFile> deleteManifests = 
snapshot.deleteManifests(table().io());
+
+                  ManifestGroup manifestGroup =
+                      new ManifestGroup(table().io(), dataManifests, 
deleteManifests)
+                          .specsById(table().specs())
+                          .caseSensitive(isCaseSensitive())
+                          .select(scanColumns())
+                          .filterData(filter())
+                          .columnsToKeepStats(columnsToKeepStats());
+
+                  if (shouldIgnoreResiduals()) {
+                    manifestGroup = manifestGroup.ignoreResiduals();
+                  }
+
+                  if (dataManifests.size() > 1 && shouldPlanWithExecutor()) {

Review Comment:
   The parallelism in `ManifestGroup::plan` is at the level of data manifest 
files, i.e., the unit of work is processing a data manifest file. So asking for 
planning with an executor service if there are many equality delete files does 
not help if there is only one data manifest file.



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to