aokolnychyi commented on code in PR #8123:
URL: https://github.com/apache/iceberg/pull/8123#discussion_r1309320715


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -95,6 +97,10 @@ public Table table() {
     return table;
   }
 
+  public FileIO io() {

Review Comment:
   Done.



##########
core/src/main/java/org/apache/iceberg/DistributedDataBatchScan.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * 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.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.ManifestEvaluator;
+import org.apache.iceberg.expressions.Projections;
+import org.apache.iceberg.expressions.ResidualEvaluator;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.metrics.ScanMetricsUtil;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.ParallelIterable;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.iceberg.util.ThreadPools;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An abstract class for batch data scans that can utilize cluster resources 
for planning.
+ *
+ * <p>This class provides common logic to create data scans that are capable 
of reading and
+ * filtering manifests remotely when the metadata size exceeds the threshold 
for local processing.
+ * Additionally, it also takes care of planning tasks locally if remote 
planning is not considered
+ * beneficial.
+ *
+ * <p>Note that this class is evolving and is subject to change even in minor 
releases.
+ */
+abstract class DistributedDataBatchScan
+    extends DataScan<BatchScan, ScanTask, ScanTaskGroup<ScanTask>> implements 
BatchScan {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(DistributedDataBatchScan.class);
+  private static final int NUM_LOCAL_CORES = 
Runtime.getRuntime().availableProcessors();
+  private static final long LOCAL_PLANNING_MAX_SIZE_PER_CORE = 128L * 1024 * 
1024; // 128 MB
+  private static final int MONITOR_POOL_SIZE = 2;
+
+  protected DistributedDataBatchScan(Table table, Schema schema, 
TableScanContext context) {
+    super(table, schema, context);
+  }
+
+  /**
+   * Returns the cluster parallelism.
+   *
+   * <p>This value indicates the maximum number of manifests that can be 
processed concurrently by
+   * the cluster. Implementations should take into account both the currently 
available processing
+   * slots and potential dynamic allocation, if applicable.
+   *
+   * <p>The remote parallelism is compared against the number of local cores 
to determine the
+   * feasibility of remote planning. This value is ignored if the planning 
mode is set explicitly.
+   */
+  protected abstract int remoteParallelism();
+
+  /** Returns the planning mode for data. */
+  protected abstract PlanningMode dataPlanningMode();
+
+  /**
+   * Controls whether defensive copies are created for remotely planned data 
files.
+   *
+   * <p>By default, this class creates defensive copies for each data file 
that is planned remotely,
+   * assuming the provided iterable can be lazy and may reuse objects. If 
unnecessary and data file
+   * objects can be used as-is, implementations can override this behavior.
+   */
+  protected boolean shouldCopyRemotelyPlannedDataFiles() {
+    return true;
+  }
+
+  /**
+   * Plans groups of data files remotely.
+   *
+   * <p>Implementations may split the matching files into any number of 
groups. This class may
+   * decide to process each group concurrently to speed up the planning 
process. In particular, it
+   * may be useful if there are equality deletes, as index lookups would 
involve comparing
+   * boundaries for equality fields and are fairly expensive.
+   *
+   * <p>If the result iterable reuses objects, {@link 
#shouldCopyRemotelyPlannedDataFiles()} must
+   * return true.
+   *
+   * @param dataManifests data manifests that may contain files matching the 
scan filter
+   * @param withColumnStats a flag whether to load column stats
+   * @return groups of data files planned remotely
+   */
+  protected abstract Iterable<CloseableIterable<DataFile>> planDataRemotely(
+      List<ManifestFile> dataManifests, boolean withColumnStats);
+
+  /** Returns the planning mode for deletes. */
+  protected abstract PlanningMode deletePlanningMode();
+
+  /**
+   * Plans deletes remotely and returns a delete file index.
+   *
+   * @param deleteManifests delete manifests that may contain files matching 
the scan filter
+   * @return a delete file index planned remotely
+   */
+  protected abstract DeleteFileIndex planDeletesRemotely(List<ManifestFile> 
deleteManifests);
+
+  @Override
+  protected CloseableIterable<ScanTask> doPlanFiles() {
+    Snapshot snapshot = snapshot();
+
+    List<ManifestFile> dataManifests = findMatchingDataManifests(snapshot);
+    boolean planDataLocally = shouldPlanLocally(dataPlanningMode(), 
dataManifests);
+
+    List<ManifestFile> deleteManifests = findMatchingDeleteManifests(snapshot);
+    boolean planDeletesLocally = shouldPlanLocally(deletePlanningMode(), 
deleteManifests);
+    boolean mayHaveEqualityDeletes = deleteManifests.size() > 0 && 
mayHaveEqualityDeletes(snapshot);
+
+    boolean shouldCopyDataFiles =
+        planDataLocally
+            || shouldCopyRemotelyPlannedDataFiles()
+            || (mayHaveEqualityDeletes && !shouldReturnColumnStats());
+
+    if (planDataLocally && planDeletesLocally) {
+      LOG.info("Planning data and deletes locally for table {}", 
table().name());
+      ManifestGroup manifestGroup = newManifestGroup(dataManifests, 
deleteManifests);
+      return manifestGroup.planFilesAsScanTasks();
+    }
+
+    ExecutorService monitorPool = newMonitorPool();
+
+    CompletableFuture<DeleteFileIndex> deletesFuture =
+        newDeletesFuture(deleteManifests, planDeletesLocally, monitorPool);
+
+    CompletableFuture<Iterable<CloseableIterable<DataFile>>> dataFuture =
+        newDataFuture(dataManifests, planDataLocally, mayHaveEqualityDeletes, 
monitorPool);
+
+    try {
+      Iterable<CloseableIterable<ScanTask>> fileTasks =
+          toFileTasks(dataFuture, deletesFuture, shouldCopyDataFiles);
+
+      if (shouldPlanWithExecutor() && (planDataLocally || 
mayHaveEqualityDeletes)) {
+        return new ParallelIterable<>(fileTasks, planExecutor());
+      } else {
+        return CloseableIterable.concat(fileTasks);
+      }
+
+    } catch (CompletionException e) {
+      dataFuture.cancel(true /* may interrupt */);
+      deletesFuture.cancel(true /* may interrupt */);
+      throw new RuntimeException("Failed to plan files", e);
+
+    } finally {
+      monitorPool.shutdown();
+    }
+  }
+
+  @Override
+  public CloseableIterable<ScanTaskGroup<ScanTask>> planTasks() {
+    return TableScanUtil.planTaskGroups(
+        planFiles(), targetSplitSize(), splitLookback(), splitOpenFileCost());
+  }
+
+  private List<ManifestFile> findMatchingDataManifests(Snapshot snapshot) {
+    List<ManifestFile> dataManifests = snapshot.dataManifests(io());
+    scanMetrics().totalDataManifests().increment(dataManifests.size());
+
+    List<ManifestFile> matchingDataManifests = filterManifests(dataManifests);
+    int skippedDataManifestsCount = dataManifests.size() - 
matchingDataManifests.size();
+    scanMetrics().skippedDataManifests().increment(skippedDataManifestsCount);
+
+    return matchingDataManifests;
+  }
+
+  private List<ManifestFile> findMatchingDeleteManifests(Snapshot snapshot) {
+    List<ManifestFile> deleteManifests = snapshot.deleteManifests(io());
+    scanMetrics().totalDeleteManifests().increment(deleteManifests.size());
+
+    List<ManifestFile> matchingDeleteManifests = 
filterManifests(deleteManifests);
+    int skippedDeleteManifestsCount = deleteManifests.size() - 
matchingDeleteManifests.size();
+    
scanMetrics().skippedDeleteManifests().increment(skippedDeleteManifestsCount);
+
+    return matchingDeleteManifests;
+  }
+
+  private List<ManifestFile> filterManifests(List<ManifestFile> manifests) {
+    Map<Integer, ManifestEvaluator> evalCache = 
specCache(this::newManifestEvaluator);
+
+    return manifests.stream()
+        .filter(manifest -> manifest.hasAddedFiles() || 
manifest.hasExistingFiles())
+        .filter(manifest -> 
evalCache.get(manifest.partitionSpecId()).eval(manifest))
+        .collect(Collectors.toList());
+  }
+
+  protected boolean shouldPlanLocally(PlanningMode mode, List<ManifestFile> 
manifests) {
+    if (context().planWithCustomizedExecutor()) {
+      return true;
+    }
+
+    switch (mode) {
+      case LOCAL:
+        return true;
+
+      case DISTRIBUTED:
+        return manifests.size() <= 1;
+
+      case AUTO:
+        return remoteParallelism() <= NUM_LOCAL_CORES

Review Comment:
   Done.



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