RussellSpitzer commented on a change in pull request #1421: URL: https://github.com/apache/iceberg/pull/1421#discussion_r505826847
########## File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.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.actions; + +import com.github.benmanes.caffeine.cache.CacheLoader; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.LoadingCache; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DeleteFileIndex; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScanContext; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.InclusiveMetricsEvaluator; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.TableScanUtil; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PlanScanAction extends BaseAction<CloseableIterable<CombinedScanTask>> { + public static final String DISTRIBUTED_SCAN_PLANNING = "iceberg.plan_distributed"; + private static final Logger LOG = LoggerFactory.getLogger(PlanScanAction.class); + + private final Table table; + private final SparkSession spark; + private final JavaSparkContext jsc; + private final TableOperations ops; + private final Schema schema; + + private TableScanContext context; + + public PlanScanAction(SparkSession spark, Table table) { + this.table = table; + this.spark = spark; + this.jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.schema = table.schema(); + this.ops = ((HasTableOperations) table).operations(); + this.context = new TableScanContext(); + } + + public PlanScanAction withContext(TableScanContext newContext) { + this.context = newContext; + return this; + } + + @Override + protected Table table() { + return table; + } + + @Override + public CloseableIterable<CombinedScanTask> execute() { + LOG.debug("Preparing distributed planning of scan for {}", table); + CloseableIterable<CombinedScanTask> result = planTasks(); + LOG.debug("Planning complete"); + return result; + } + + protected CloseableIterable<CombinedScanTask> planTasks() { + Map<String, String> options = context.options(); + long splitSize; + if (options.containsKey(TableProperties.SPLIT_SIZE)) { + splitSize = Long.parseLong(options.get(TableProperties.SPLIT_SIZE)); + } else { + splitSize = ops.current().propertyAsLong(TableProperties.SPLIT_SIZE, TableProperties.SPLIT_SIZE_DEFAULT); + } + int lookback; + if (options.containsKey(TableProperties.SPLIT_LOOKBACK)) { + lookback = Integer.parseInt(options.get(TableProperties.SPLIT_LOOKBACK)); + } else { + lookback = ops.current().propertyAsInt( + TableProperties.SPLIT_LOOKBACK, TableProperties.SPLIT_LOOKBACK_DEFAULT); + } + long openFileCost; + if (options.containsKey(TableProperties.SPLIT_OPEN_FILE_COST)) { + openFileCost = Long.parseLong(options.get(TableProperties.SPLIT_OPEN_FILE_COST)); + } else { + openFileCost = ops.current().propertyAsLong( + TableProperties.SPLIT_OPEN_FILE_COST, TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + } + + CloseableIterable<FileScanTask> fileScanTasks = planFiles(); + CloseableIterable<FileScanTask> splitFiles = TableScanUtil.splitFiles(fileScanTasks, splitSize); + return TableScanUtil.planTasks(splitFiles, splitSize, lookback, openFileCost); + } + + private Snapshot snapshot() { + return context.snapshotId() != null ? + ops.current().snapshot(context.snapshotId()) : + ops.current().currentSnapshot(); + } + + public CloseableIterable<FileScanTask> planFiles() { + // Create a dataframe of all DataFile entries + String dataFilesMetadataTable = metadataTableName(MetadataTableType.ENTRIES); + Dataset<Row> manifestEntries = + spark.read() + .format("iceberg") + .load(dataFilesMetadataTable); + + // Todo pushdown filters to ManifestEntriesTable + // Read entries which are not deleted and are datafiles and not delete files + Dataset<Row> dataFileEntries = manifestEntries + .filter(manifestEntries.col("data_file").getField(DataFile.CONTENT.name()).equalTo(0)) // Only DataFiles + .filter(manifestEntries.col("status").notEqual(2)); // No Deleted Files Review comment: Yes but within the private ManifestEntry class which is why we have the magic here ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
