aokolnychyi commented on code in PR #8123: URL: https://github.com/apache/iceberg/pull/8123#discussion_r1308223239
########## spark/v3.4/spark/src/main/java/org/apache/iceberg/SparkDistributedDataBatchScan.java: ########## @@ -0,0 +1,222 @@ +/* + * 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.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.shaded.com.google.common.collect.Iterables; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.ClosingIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.JobGroupUtils; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.actions.ManifestFileBean; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.SparkSession; + +public class SparkDistributedDataBatchScan extends DistributedDataBatchScan { + + private static final Joiner COMMA = Joiner.on(','); + private static final String DELETE_PLANNING_JOB_GROUP_ID = "DELETE-PLANNING"; + private static final String DATA_PLANNING_JOB_GROUP_ID = "DATA-PLANNING"; + + private final SparkSession spark; + private final JavaSparkContext sparkContext; + private final SparkReadConf readConf; + + private Broadcast<Table> tableBroadcast = null; + + public SparkDistributedDataBatchScan(SparkSession spark, Table table, SparkReadConf readConf) { + this(spark, table, readConf, table.schema(), TableScanContext.empty()); + } + + private SparkDistributedDataBatchScan( + SparkSession spark, + Table table, + SparkReadConf readConf, + Schema schema, + TableScanContext context) { + super(table, schema, context); + this.spark = spark; + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.readConf = readConf; + } + + @Override + protected BatchScan newRefinedScan( + Table newTable, Schema newSchema, TableScanContext newContext) { + return new SparkDistributedDataBatchScan(spark, newTable, readConf, newSchema, newContext); + } + + @Override + protected int remoteParallelism() { + return readConf.parallelism(); + } + + @Override + protected PlanningMode dataPlanningMode() { + return readConf.dataPlanningMode(); + } + + @Override + protected PlanningMode deletePlanningMode() { + return readConf.deletePlanningMode(); + } + + @Override + protected boolean shouldCopyRemotelyPlannedDataFiles() { + return false; + } + + @Override + protected Iterable<CloseableIterable<DataFile>> planDataRemotely( + List<ManifestFile> dataManifests, boolean withColumnStats) { + JobGroupInfo info = new JobGroupInfo(DATA_PLANNING_JOB_GROUP_ID, jobDesc("data")); + return withJobGroupInfo(info, () -> doPlanDataRemotely(dataManifests, withColumnStats)); + } + + private Iterable<CloseableIterable<DataFile>> doPlanDataRemotely( + List<ManifestFile> dataManifests, boolean withColumnStats) { + scanMetrics().scannedDataManifests().increment(dataManifests.size()); + + JavaRDD<DataFile> dataFileRDD = + sparkContext + .parallelize(toBeans(dataManifests), dataManifests.size()) + .flatMap(new ReadDataManifest(tableBroadcast(), context(), withColumnStats)); + List<List<DataFile>> dataFileGroups = collectPartitions(dataFileRDD); + + return Iterables.transform(dataFileGroups, CloseableIterable::withNoopClose); + } + + @Override + protected DeleteFileIndex planDeletesRemotely(List<ManifestFile> deleteManifests) { + JobGroupInfo info = new JobGroupInfo(DELETE_PLANNING_JOB_GROUP_ID, jobDesc("deletes")); + return withJobGroupInfo(info, () -> doPlanDeletesRemotely(deleteManifests)); + } + + private DeleteFileIndex doPlanDeletesRemotely(List<ManifestFile> deleteManifests) { + scanMetrics().scannedDeleteManifests().increment(deleteManifests.size()); + + List<DeleteFile> deleteFiles = + sparkContext + .parallelize(toBeans(deleteManifests), deleteManifests.size()) + .flatMap(new ReadDeleteManifest(tableBroadcast(), context())) + .collect(); + + return DeleteFileIndex.builderFor(deleteFiles) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .scanMetrics(scanMetrics()) + .build(); + } + + private <T> T withJobGroupInfo(JobGroupInfo info, Supplier<T> supplier) { + return JobGroupUtils.withJobGroupInfo(spark, info, supplier); + } + + private String jobDesc(String type) { + List<String> options = Lists.newArrayList(); + options.add("snapshot_id=" + snapshot().snapshotId()); + String optionsAsString = COMMA.join(options); + return String.format("Planning %s (%s) for %s", type, optionsAsString, table().name()); + } + + private List<ManifestFileBean> toBeans(List<ManifestFile> manifests) { + return manifests.stream().map(ManifestFileBean::fromManifest).collect(Collectors.toList()); + } + + private Broadcast<Table> tableBroadcast() { + if (tableBroadcast == null) { + Table serializableTable = SerializableTableWithSize.copyOf(table()); + this.tableBroadcast = sparkContext.broadcast(serializableTable); + } + + return tableBroadcast; + } + + private <T> List<List<T>> collectPartitions(JavaRDD<T> rdd) { + int[] partitionIds = IntStream.range(0, rdd.getNumPartitions()).toArray(); + return Arrays.asList(rdd.collectPartitions(partitionIds)); + } + + private static class ReadDataManifest implements FlatMapFunction<ManifestFileBean, DataFile> { + + private final Broadcast<Table> table; + private final Expression filter; + private final boolean withStats; + private final boolean isCaseSensitive; + + ReadDataManifest(Broadcast<Table> table, TableScanContext context, boolean withStats) { + this.table = table; + this.filter = context.rowFilter(); + this.withStats = withStats; + this.isCaseSensitive = context.caseSensitive(); + } + + @Override + public Iterator<DataFile> call(ManifestFileBean manifest) throws Exception { + FileIO io = table.getValue().io(); Review Comment: Good catch. ########## 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: Yep, that makes more sense. -- 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]
