rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499793670



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.BaseTableScan;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataTableScan;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.DeleteFileIndex;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.IncrementalDataTableScan;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PartitionSpecParser;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.events.ScanEvent;
+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.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>> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(PlanScanAction.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  private final Table table;
+  private final TableScan scan;
+  private final SparkSession spark;
+  private final Snapshot snapshot;
+
+  private PlanScanAction(SparkSession spark, Table table, TableScan scan) {
+    this.spark = spark;
+    this.table = table;
+    this.scan = scan;
+    this.snapshot = scan.snapshot();
+  }
+
+  public PlanScanAction(SparkSession spark, Table table, DataTableScan scan) {
+    this(spark, table, (TableScan) scan);
+  }
+
+  @Override
+  protected Table table() {
+    return table;
+  }
+
+  @Override
+  public CloseableIterable<CombinedScanTask> execute() {
+    Map<String, String> options = ((BaseTableScan) scan).options();
+    TableMetadata meta = ((BaseTableScan) scan).tableOps().current(); // TODO 
maybe pass through metadata instead
+    long splitSize;
+    if (options.containsKey(TableProperties.SPLIT_SIZE)) {
+      splitSize = Long.parseLong(options.get(TableProperties.SPLIT_SIZE));
+    } else {
+      splitSize = meta.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 = meta.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 = meta.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 CloseableIterable<FileScanTask> planFiles() {
+    if (snapshot != null) {
+      LOG.info("Using distributed planning for scanning table {} snapshot {} 
created at {} with filter {}",
+          scan.table(), snapshot.snapshotId(), snapshot.timestampMillis(), 
scan.filter());
+      Listeners.notifyAll(
+          new ScanEvent(scan.table().toString(), snapshot.snapshotId(), 
scan.filter(), scan.schema()));
+      return planFilesFor();
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private CloseableIterable<FileScanTask> planFilesFor() {
+    // This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles();
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do 
distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  private DeleteFileIndex buildDeleteFileIndex() {
+    // Build delete index locally
+    List<ManifestFile> deleteManifests = snapshot.deleteManifests();
+    DeleteFileIndex.Builder deleteFileIndexBuilder = 
DeleteFileIndex.builderFor(table.io(), deleteManifests);
+    deleteFileIndexBuilder.caseSensitive(scan.isCaseSensitive());
+    deleteFileIndexBuilder.specsById(table.specs());
+    deleteFileIndexBuilder.filterData(scan.filter());
+    return deleteFileIndexBuilder.build();
+  }
+
+  private CloseableIterable<FileScanTask> planDataTableScanFiles() {
+    // TODO Currently this approach reads all manifests, no manifest filtering 
- Maybe through pushdowns or table read
+    // options?
+
+
+    // 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
+
+    // Handle Incremental Scans
+    if (scan instanceof IncrementalDataTableScan) {
+      IncrementalDataTableScan incScan = (IncrementalDataTableScan) scan;
+      List<Snapshot> snapshots = 
IncrementalDataTableScan.snapshotsWithin(table, incScan.fromSnapshotId(),
+          incScan.toSnapshotId());
+      List<Long> validSnapshotIds = 
snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      dataFileEntries = dataFileEntries
+          
.filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()));
+    }
+
+    // Build up evaluators and filters for Metrics and Partition values
+    final Expression scanFilter = scan.filter();
+    final boolean ignoreResiduals = scan.shouldIgnoreResiduals();
+    final boolean isCaseSensitive = scan.isCaseSensitive();
+
+    Map<Integer, PartitionSpec> specsById = scan.table().specs();
+    JavaSparkContext jsc = 
JavaSparkContext.fromSparkContext(spark.sparkContext());
+
+    // Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = 
ImmutableMap.builder();
+    specsById.entrySet().stream().forEach(entry ->
+        evalMapBuilder.put(entry.getKey(),
+            new Evaluator(entry.getValue().partitionType(),
+                Projections.inclusive(entry.getValue(), 
isCaseSensitive).project(scanFilter))));
+
+    Map<Integer, Evaluator> partitionEvaluatorsById = evalMapBuilder.build();
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = 
jsc.broadcast(partitionEvaluatorsById);
+
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator =  
jsc.broadcast(
+        new InclusiveMetricsEvaluator(scan.schema(), scanFilter, 
isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    final Broadcast<Map<Integer, PartitionSpec>> broadcastSpecsById = 
jsc.broadcast(specsById);
+
+    Types.StructType partitionStruct = 
DataFile.getType(scan.table().spec().partitionType());
+    StructType dataFileSchema = (StructType) 
dataFileEntries.schema().apply("data_file").dataType();
+
+    // Evaluate all files based on their partition info and collect the rows 
back locally
+    Dataset<Row> scanTaskDataset = dataFileEntries.mapPartitions(
+        (MapPartitionsFunction<Row, Row>) it -> {
+          SparkDataFile container = new SparkDataFile(partitionStruct, 
dataFileSchema);
+          return Streams.stream(it)
+              .filter(row -> {
+                Row dataFile = row.getAs("data_file");
+                SparkDataFile file = container.wrap(dataFile);
+                return 
broadcastPartitionEvaluators.getValue().get(file.specId()).eval(file.partition())
 &&
+                    broadcastMetricsEvaluator.getValue().eval(file);
+              }).iterator();
+        }, RowEncoder.apply(dataFileEntries.schema()));

Review comment:
       I think it would be better long term if the partition expression was 
converted to a Spark or SQL filter instead of evaluating it directly. That way, 
Spark can push the filters down to the metadata table scan when for when it 
does support filter pushdown using the manifest list partition summaries.
   
   That would probably also make execution a little faster since we assume that 
Spark's filter evaluation is faster because it uses codegen.




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

Reply via email to