You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/10/26 14:11:42 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #1421: Add a Parallelized Spark Job Planning Path

RussellSpitzer opened a new pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421


   This is the second of two WIPs for parallelizing Spark Read Job Planning
   
   The other is located at https://github.com/apache/iceberg/pull/1420
   
   To parallelize the creation of TableScanTasks, we use the
   metadata tables to get a listing of DataFiles and do filtering in
   spark before starting the scan job. Once the correct datafiles are
   identified, scan tasks are created and returned.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499794573



##########
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);

Review comment:
       Yeah, I'm pushing the complexity bounds pretty hard here. I'll move it out like the deletefileindex building and friends




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rymurr commented on pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-877806701


   Hey @RussellSpitzer and @aokolnychyi what was the status of this PR? We could use some of the changes here to core/api for our planning. Is this close to merging once conflicts are fixed?


-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499681933



##########
File path: core/src/main/java/org/apache/iceberg/V1Metadata.java
##########
@@ -350,6 +351,8 @@ public Object get(int pos) {
           return wrapped.keyMetadata();
         case 12:
           return wrapped.splitOffsets();
+        case 13:
+          return specId();

Review comment:
       Required to keep tests passing, when fields are not named in the schema avro makes up it's own field name which breaks some tests.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] holdenk commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r506664772



##########
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:
       Gotcha, makes 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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499776993



##########
File path: core/src/main/java/org/apache/iceberg/BaseFileScanTask.java
##########
@@ -29,7 +29,7 @@
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
 
-class BaseFileScanTask implements FileScanTask {
+public class BaseFileScanTask implements FileScanTask {

Review comment:
       I think maybe we should have a factory method somewhere in the public API instead? We don't necessarily want to expose everything 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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499679763



##########
File path: core/src/main/java/org/apache/iceberg/DeleteFileIndex.java
##########
@@ -62,7 +62,7 @@
  * Use {@link #builderFor(FileIO, Iterable)} to construct an index, and {@link #forDataFile(long, DataFile)} or
  * {@link #forEntry(ManifestEntry)} to get the the delete files to apply to a given data file.
  */
-class DeleteFileIndex {
+public class DeleteFileIndex {

Review comment:
       Public again because we need to access this class from the Spark package in order to do our planning




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499920578



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);

Review comment:
       One awkward thing I've just hit as I've just about finished this refactor, If we make the decision when we are building the scan and not when we call planFiles, we can't fallback from distributed to not distributed if it doesn't work, unless we maintain two scan objects




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483688683



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {

Review comment:
       Thinking out loud: will it be appropriate to a have an action for planning?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511173231



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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(

Review comment:
       Sure we are projecting within this map partition already, (note that we only take the data file out of the row) but I can move the projection slightly above so catalyst can do the project and we won't do it with scala objects.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-709563046


   There are a few 
   
   > Oh yeah this is a big one :) I'll try and look at it tomorrow afternoon. Is there a part of it where you would like me to focus?
   
   I think we are mainly looking for any ways to reduce the amount of internals that are being opened up, and also thinking about any places where there are potential simplifications. If you could think of a cleaner way to get that "specID" property into our DataFiles table that would be awesome too, it's definitely the most hacky part of this imho.
   
   We were punting on adaptive implementation of distributed planning till a later ticket as well as pushing down filters directly into our Metadata tables. 


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510415243



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java
##########
@@ -41,15 +42,32 @@
 import org.apache.spark.sql.internal.SQLConf;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
 
+@RunWith(Parameterized.class)
 public abstract class TestSparkReaderDeletes extends DeleteReadTests {
 
   private static TestHiveMetastore metastore = null;
   protected static SparkSession spark = null;
   protected static HiveCatalog catalog = null;
 
+  @Parameterized.Parameters(name = "Distributed Planning {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { false },
+        new Object[] { true },
+    };
+  }

Review comment:
       Is there no way to collapse this into a single `Object[]`?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499783725



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);

Review comment:
       What about making `PlanScanAction` implement `TableScan` instead of passing it a `TableScan`? Then you'd instantiate it above in place of `scan` and configure it the same way. That may avoid the need to change so much of the public API to get data from the scan.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499794203



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);

Review comment:
       This is a pretty fun Idea, let me try that




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] holdenk commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505809527



##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
+    List<Long> snapshotIds = SnapshotUtil.snapshotIdsBetween(table, fromSnapshotId, toSnapshotId);
+    List<Snapshot> snapshots = Lists.newArrayList();
+    for (Long snapshotId : snapshotIds) {
+      Snapshot snapshot = table.snapshot(snapshotId);
+      // for now, incremental scan supports only appends
+      if (snapshot.operation().equals(DataOperations.APPEND)) {
+        snapshots.add(snapshot);
+      } else if (snapshot.operation().equals(DataOperations.OVERWRITE)) {
+        throw new UnsupportedOperationException(
+            String.format("Found %s operation, cannot support incremental data in snapshots (%s, %s]",
+                DataOperations.OVERWRITE, fromSnapshotId, toSnapshotId));
+      }
+    }
+    return snapshots;
+  }
+
+  public static void validateSnapshotIdsRefinement(long newFromSnapshotId, long newToSnapshotId, Table table,
+                                                   TableScanContext context) {
+    Set<Long> snapshotIdsRange = Sets.newHashSet(
+        SnapshotUtil.snapshotIdsBetween(table, context.fromSnapshotId(), context.toSnapshotId()));

Review comment:
       This seems like it could make a large set. Is that the case? Is it avoidable?

##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
+    List<Long> snapshotIds = SnapshotUtil.snapshotIdsBetween(table, fromSnapshotId, toSnapshotId);
+    List<Snapshot> snapshots = Lists.newArrayList();
+    for (Long snapshotId : snapshotIds) {
+      Snapshot snapshot = table.snapshot(snapshotId);
+      // for now, incremental scan supports only appends
+      if (snapshot.operation().equals(DataOperations.APPEND)) {
+        snapshots.add(snapshot);
+      } else if (snapshot.operation().equals(DataOperations.OVERWRITE)) {
+        throw new UnsupportedOperationException(
+            String.format("Found %s operation, cannot support incremental data in snapshots (%s, %s]",
+                DataOperations.OVERWRITE, fromSnapshotId, toSnapshotId));
+      }
+    }
+    return snapshots;
+  }
+
+  public static void validateSnapshotIdsRefinement(long newFromSnapshotId, long newToSnapshotId, Table table,

Review comment:
       This is validating that the new from/to is a subset of the current context range yes?

##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
+    List<Long> snapshotIds = SnapshotUtil.snapshotIdsBetween(table, fromSnapshotId, toSnapshotId);
+    List<Snapshot> snapshots = Lists.newArrayList();
+    for (Long snapshotId : snapshotIds) {
+      Snapshot snapshot = table.snapshot(snapshotId);
+      // for now, incremental scan supports only appends
+      if (snapshot.operation().equals(DataOperations.APPEND)) {
+        snapshots.add(snapshot);
+      } else if (snapshot.operation().equals(DataOperations.OVERWRITE)) {
+        throw new UnsupportedOperationException(
+            String.format("Found %s operation, cannot support incremental data in snapshots (%s, %s]",
+                DataOperations.OVERWRITE, fromSnapshotId, toSnapshotId));
+      }
+    }
+    return snapshots;
+  }
+
+  public static void validateSnapshotIdsRefinement(long newFromSnapshotId, long newToSnapshotId, Table table,
+                                                   TableScanContext context) {
+    Set<Long> snapshotIdsRange = Sets.newHashSet(
+        SnapshotUtil.snapshotIdsBetween(table, context.fromSnapshotId(), context.toSnapshotId()));
+    // since snapshotIdsBetween return ids in range (fromSnapshotId, toSnapshotId]
+    snapshotIdsRange.add(context.fromSnapshotId());
+    Preconditions.checkArgument(
+        snapshotIdsRange.contains(newFromSnapshotId),
+        "from snapshot id %s not in existing snapshot ids range (%s, %s]",
+        newFromSnapshotId, context.fromSnapshotId(), newToSnapshotId);
+    Preconditions.checkArgument(
+        snapshotIdsRange.contains(newToSnapshotId),
+        "to snapshot id %s not in existing snapshot ids range (%s, %s]",
+        newToSnapshotId, context.fromSnapshotId(), context.toSnapshotId());
+  }
+
+  public static void validateSnapshotIds(Table table, long fromSnapshotId, long toSnapshotId) {

Review comment:
       This validates the snapshots are present in this table yes?

##########
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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) new BaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString,
+          cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))

Review comment:
       Could we use a range here to save sending all of the validSnapshotIDs accross the wire? Or are there going to be intermediate files with snapshotids we need to exclude?
   (I'm assuming we probably need to exclude intermediate files but just making sure since I'm new to this code).

##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {

Review comment:
       Might it be good to add some javadoc here? I could see someone calling this when they really want `snapshotIdsBetween`

##########
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:
       Are there enums of these somewhere?

##########
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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) new BaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString,
+          cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))
+          .filter(dataFileEntries.col("status").equalTo(1)); // Added files only

Review comment:
       enum?

##########
File path: core/src/main/java/org/apache/iceberg/util/TableScanUtil.java
##########
@@ -57,4 +57,6 @@ public static boolean hasDeletes(FileScanTask task) {
             splitFiles),
         BaseCombinedScanTask::new);
   }
+

Review comment:
       Is this intentional?

##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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("Distributed 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))
+        .filter(manifestEntries.col("status").notEqual(2)); // not deleted
+
+    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()));
+    }
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    // 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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache =
+        Caffeine.newBuilder().build((CacheLoader<Integer, SpecCacheEntry> & Serializable) specId -> {
+          PartitionSpec spec = broadcastSpecsById.getValue().get(specId);
+          Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : scanFilter;
+          return new SpecCacheEntry(SchemaParser.toJson(spec.schema()), PartitionSpecParser.toJson(spec),
+              ResidualEvaluator.of(spec, filter, isCaseSensitive));
+        });
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {

Review comment:
       scnaTaskDataset is always going to be relatively small in return value right?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499778295



##########
File path: core/src/main/java/org/apache/iceberg/DeleteFileIndex.java
##########
@@ -62,7 +62,7 @@
  * Use {@link #builderFor(FileIO, Iterable)} to construct an index, and {@link #forDataFile(long, DataFile)} or
  * {@link #forEntry(ManifestEntry)} to get the the delete files to apply to a given data file.
  */
-class DeleteFileIndex {
+public class DeleteFileIndex {

Review comment:
       This should be okay, as long as we aren't exposing `ManifestEntry`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511147503



##########
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.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.ScanTasks;
+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);

Review comment:
       extracting the utility method formatTimestamp to TableScanUtils and mimicking TableScan debug line




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483712254



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));
+
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    //Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = new 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);

Review comment:
       If we end up merging everything into a single filter function I think we end up just using a single loading cache like below where we do one lookup to get all the relative evaluators and the cache is owned by the executor. Probably slightly more efficient to since some executors won't see information from all partition specs...?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511107203



##########
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.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.ScanTasks;
+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";

Review comment:
       Sounds good to me




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511128454



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();

Review comment:
       Yeah let's prompt inside the DeleteFile Index forFile so we hit in all uses of DeleteIndex, I'll make that change in another PR




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511064483



##########
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.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.ScanTasks;
+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);

Review comment:
       Would it make sense to make it info and log additional info as we do in the non-distributed mode?
   
   Here is what we have there:
   
   ```
         LOG.info("Scanning table {} snapshot {} created at {} with filter {}", table,
             snapshot.snapshotId(), formatTimestampMillis(snapshot.timestampMillis()),
             context.rowFilter());
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511081736



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);

Review comment:
       I think the name of this function is confusing.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505835042



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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("Distributed 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))
+        .filter(manifestEntries.col("status").notEqual(2)); // not deleted
+
+    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()));
+    }
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    // 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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache =
+        Caffeine.newBuilder().build((CacheLoader<Integer, SpecCacheEntry> & Serializable) specId -> {
+          PartitionSpec spec = broadcastSpecsById.getValue().get(specId);
+          Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : scanFilter;
+          return new SpecCacheEntry(SchemaParser.toJson(spec.schema()), PartitionSpecParser.toJson(spec),
+              ResidualEvaluator.of(spec, filter, isCaseSensitive));
+        });
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {

Review comment:
       It will be no larger than one we would have to make locally, so yes there is a possibility for an OOM here, but no more so than during the normal planning path. 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r504983333



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -46,18 +47,38 @@
   private final int upperBoundsPosition;
   private final int keyMetadataPosition;
   private final int splitOffsetsPosition;
+  private final int specIdPosition;
   private final Type lowerBoundsType;
   private final Type upperBoundsType;
   private final Type keyMetadataType;
 
   private final SparkStructLike wrappedPartition;
+  private final Types.StructType partitionStruct;
   private Row wrapped;
 
+  private static final StructLike EMPTY_PARTITION_INFO = new StructLike() {
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return null;

Review comment:
       Sgtm, mirrored this in the "set" command, might as well throw in both.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499782656



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);
     }
-
     return tasks;
   }
 
+  private List<CombinedScanTask> planScan(TableScan scan) {
+    // TODO Need to only use distributed planner for supported implementations and add some heuristics about when
+    //  to use

Review comment:
       Initially, we could use a table property to enable this.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499783725



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);

Review comment:
       What about making `PlanScanAction` implement `TableScan` instead of passing it a `TableScan`? Then you'd instantiate it above in place of `scan` and configure it the same way.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511084669



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) ScanTasks
+          .createBaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString, cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))
+          .filter(dataFileEntries.col("status").equalTo(1)); // Added files only
+    } else if (context.snapshotId() != null) {
+      LOG.debug("Planning scan at snapshot {}", context.snapshotId());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").equalTo(context.snapshotId()));
+    } else {
+      return dataFileEntries;
+    }
+  }
+
+  private LoadingCache<Integer, SpecCacheEntry> buildSpecCache() {
+    return Caffeine.newBuilder().build((CacheLoader<Integer, SpecCacheEntry> & Serializable) specId -> {
+      PartitionSpec spec = table().specs().get(specId);
+      Expression filter = context.ignoreResiduals() ? Expressions.alwaysTrue() : context.rowFilter();
+      return new SpecCacheEntry(SchemaParser.toJson(spec.schema()), PartitionSpecParser.toJson(spec),
+          ResidualEvaluator.of(spec, filter, context.caseSensitive()));
+    });
+  }
+
+  private Broadcast<Map<Integer, Evaluator>> buildPartitionEvaluators() {
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = ImmutableMap.builder();
+    boolean caseSensitive = context.caseSensitive();
+    Expression filter = context.rowFilter();
+    table.specs().entrySet().stream().forEach(entry ->

Review comment:
       nit: we can use `entrySet().forEach()` directly




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499681020



##########
File path: core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java
##########
@@ -106,7 +106,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche
     return new IncrementalDataTableScan(ops, table, schema, context);
   }
 
-  private static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {

Review comment:
       We only really need this method to be public so we can determine the snapshots to be used in the scan




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499780029



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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

Review comment:
       I think we should try to avoid making it public.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] holdenk commented on pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-746731138


   I see this is out of date with the dev branch, are you still working on this?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-703735285


   Alright crew, this WIP is fully operational and I would be glad to hear any feedback or comments you have on it, Please take a look at your convience -
   
   CC: @aokolnychyi , @rdblue , @kbendick  + Anyone else who would like to be involved


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499780029



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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

Review comment:
       I think we should try to avoid making it public. We want the API to be complete enough that you can build this kind of thing, I think. I didn't really think about the use case of copying a scan, but I think it does make some 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.

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511173231



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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(

Review comment:
       Sure we are projecting within this map partition already, (note that we only take the data file out of the row) but I can move the projection slightly above so catalyst can do the project and we won't do it with java objects.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511081736



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);

Review comment:
       I think the name of this function is confusing. It does not only handle incremental scans, right?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505826084



##########
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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) new BaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString,
+          cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))

Review comment:
       Unfortunately they are not monotonic, this needs to go as a set :/




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511083024



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) ScanTasks
+          .createBaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString, cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))
+          .filter(dataFileEntries.col("status").equalTo(1)); // Added files only
+    } else if (context.snapshotId() != null) {
+      LOG.debug("Planning scan at snapshot {}", context.snapshotId());
+      return dataFileEntries

Review comment:
       I don't think this is correct. It is not really an incremental scan. It is time travelling. We have to read all files that were valid at that snapshot, not those that were created in that snapshot.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511153171



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Yep, it's needed for Distributed Execution, not distributed planning




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] holdenk commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r509471171



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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;

Review comment:
       That makes 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.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499779393



##########
File path: core/src/main/java/org/apache/iceberg/V1Metadata.java
##########
@@ -213,7 +213,8 @@ static Schema wrapFileSchema(Types.StructType fileSchema) {
         DataFile.LOWER_BOUNDS,
         DataFile.UPPER_BOUNDS,
         DataFile.KEY_METADATA,
-        DataFile.SPLIT_OFFSETS
+        DataFile.SPLIT_OFFSETS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       This is the v1 write schema. Since we don't need to write spec IDs into manifests, I don't think that we need to add this here or to update to add it below.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511069508



##########
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.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.ScanTasks;
+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");

Review comment:
       In some places, we log how much time it took to commit. Shall we log with INFO how much time it took to plan? I know we should invest in a metrics framework but we need at least something for now, I guess.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r489133873



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));
+
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    //Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = new 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();

Review comment:
       If we go this route, would it be possible to make these metrics appear in the spark ui or in the history server ui? Possibly we consider metrics that convert to the native framework's accumulators etc that will print in the UI.
   
   I'm huge on metrics.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499794994



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Got it. But are those really serialized back? The filtering currently wraps and filters, but the original `Row` objects are returned to Spark. I don't think we are serializing `SparkDataFile` if we construct tasks on the driver.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511083935



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) ScanTasks
+          .createBaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString, cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))
+          .filter(dataFileEntries.col("status").equalTo(1)); // Added files only
+    } else if (context.snapshotId() != null) {
+      LOG.debug("Planning scan at snapshot {}", context.snapshotId());
+      return dataFileEntries

Review comment:
       If scanContext has set a specific snapshot id, we have to set it [there](https://github.com/apache/iceberg/pull/1421/files#r511081363). 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511173370



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();

Review comment:
       Added here, and I will submit another pr to do it as well.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499692064



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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("Distributed 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

Review comment:
       Currently the scan on ManifestEntries handles no pushdowns, which means every record must be serialized and every manifest must be read when doing distributed planning.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-706341597


   @rdblue Redid a bunch of refactors as we talked about, I also switched the testing to use parameterized versions of a bunch of already existing tests. I believe this does pretty good test coverage of the feature. I'll try to run some benchmarks in the near future.


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r485984770



##########
File path: core/src/main/java/org/apache/iceberg/DataFilesTable.java
##########
@@ -120,7 +127,8 @@ protected long targetSplitSize(TableOperations ops) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = io;
       this.manifest = manifest;
-      this.schema = schema;
+      //Ignore SpecID when reading

Review comment:
       Is this comment allowed by the linter? I thought it wasn't allowed to omit the space after `//`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511087611



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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(

Review comment:
       Can we make a projection at this step? We only need the data file in the transform?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505832258



##########
File path: core/src/main/java/org/apache/iceberg/util/TableScanUtil.java
##########
@@ -57,4 +57,6 @@ public static boolean hasDeletes(FileScanTask task) {
             splitFiles),
         BaseCombinedScanTask::new);
   }
+

Review comment:
       Nope! I'll clean that up




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483700923



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));

Review comment:
       As an alternative to using java serialization and making `SparkDataFile` serializable, we could work with `Dataset<Row>` directly and do `mapPartitions` and instantiate `SparkDataFile` and evaluators on executors, wrap Spark rows into data files, filter them, unwrap matching to rows. Then we could collect this to the driver and define a lazy transformation to `FileScanTask`.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511150089



##########
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.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.ScanTasks;
+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");

Review comment:
       Sure I'll add in milliseconds elapsed




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511081363



##########
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.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.ScanTasks;
+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 =

Review comment:
       I think Ryan's comment that was resolved below still applies here. We cannot use the current table state while reading the metadata table.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
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 private to the 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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499688014



##########
File path: core/src/main/java/org/apache/iceberg/BaseFileScanTask.java
##########
@@ -29,7 +29,7 @@
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
 
-class BaseFileScanTask implements FileScanTask {
+public class BaseFileScanTask implements FileScanTask {

Review comment:
       Needs to be public so we can build these in PlanScanAction




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rymurr commented on pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-878095874


   Forgot to add  the most important part: Shout if I can help! 
   
   cc @vvellanki @pravindra @snazy and @nastra 


-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] holdenk commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
holdenk commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r506664603



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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;

Review comment:
       Would it make sense to add a comment there indicating if changes are made there they should consider if they need to update this code too?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499788093



##########
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);

Review comment:
       Building this broadcast should probably be in a separate private method.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483692439



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //TODO Currently this approach reads all manifests, no manifest filtering - Maybe through pushdowns or table read

Review comment:
       If we go this route, we need to take into account partition predicates in `rowFilter` and apply them to manifests in the scan.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511173535



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);

Review comment:
       Switched the contents of the function to only handle incremental scans, "as of snapshot" now handled in the metadata scan as noted above.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505831083



##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
+    List<Long> snapshotIds = SnapshotUtil.snapshotIdsBetween(table, fromSnapshotId, toSnapshotId);
+    List<Snapshot> snapshots = Lists.newArrayList();
+    for (Long snapshotId : snapshotIds) {
+      Snapshot snapshot = table.snapshot(snapshotId);
+      // for now, incremental scan supports only appends
+      if (snapshot.operation().equals(DataOperations.APPEND)) {
+        snapshots.add(snapshot);
+      } else if (snapshot.operation().equals(DataOperations.OVERWRITE)) {
+        throw new UnsupportedOperationException(
+            String.format("Found %s operation, cannot support incremental data in snapshots (%s, %s]",
+                DataOperations.OVERWRITE, fromSnapshotId, toSnapshotId));
+      }
+    }
+    return snapshots;
+  }
+
+  public static void validateSnapshotIdsRefinement(long newFromSnapshotId, long newToSnapshotId, Table table,
+                                                   TableScanContext context) {
+    Set<Long> snapshotIdsRange = Sets.newHashSet(
+        SnapshotUtil.snapshotIdsBetween(table, context.fromSnapshotId(), context.toSnapshotId()));

Review comment:
       It could be large but this would only be used during incremental scans which should have bounded requests of Ids (those for a given time range) as opposed to normal scans which would just use a single snapshot id. 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499798163



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Yeah they only are serialized going out,
   We create a SparkDataFile, pack it into a FileScanTask which becomes a CombinedScanTask which we then make into a ReadTask which is an InputPartition, which ends up having to be serialized. (In Spark 3, it's slightly different in Spark2)




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499784659



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Okay, I see that it is because that is being collected inside a FileScanTask.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505830034



##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {

Review comment:
       Ah yeah I extracted these methods from IncrementalTableScan where they were private, now that they have been more exposed they need docs




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499798163



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Yeah they only are serialized going out,
   We create a SparkDataFile, pack it into a FileScanTask which becomes a CombinedScanTask which we then make into a ReadTask, which ends up having to be serialized. (In Spark 3, it's slightly different in Spark2)




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510413308



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java
##########
@@ -59,6 +63,17 @@
   );
   private static SparkSession spark = null;
 
+  private final boolean distributedPlanning;
+
+  @Parameterized.Parameters(name = "Distributed Planning = {0}")

Review comment:
       Came here to ask for `name`, was not disappointed 👍 




-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511061559



##########
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.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.ScanTasks;
+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";

Review comment:
       Is it the option users will have to specify to enable distributed planning?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499795672



##########
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);

Review comment:
       If so we need to add some tests, Currently this passes pretty much everything in Spark2 Suit and Spark 3 suite with the exception of a few tests which make sure only a single scan is done. I'll double check this when I add in a DistributedPlanning Specific test suite, rather than just trying it against all existing tests.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499689149



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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) {

Review comment:
       for type safety public entry points will only be for valid scan types




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499775791



##########
File path: api/src/main/java/org/apache/iceberg/DataFile.java
##########
@@ -80,7 +80,8 @@ static StructType getType(StructType partitionType) {
         UPPER_BOUNDS,
         KEY_METADATA,
         SPLIT_OFFSETS,
-        EQUALITY_IDS
+        EQUALITY_IDS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       Why optional?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499798709



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       So it's not that we are doing an explicit parallelizing or collecting, but when we make our Spark Partitions/InputSplits, those need to be serializable.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483692439



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //TODO Currently this approach reads all manifests, no manifest filtering - Maybe through pushdowns or table read

Review comment:
       If we go this route, we need to take into account partition predicates in `rowFilter` and apply them to manifests in the scan when we determine matching manifests.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511128848



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -46,18 +47,38 @@
   private final int upperBoundsPosition;
   private final int keyMetadataPosition;
   private final int splitOffsetsPosition;
+  private final int specIdPosition;
   private final Type lowerBoundsType;
   private final Type upperBoundsType;
   private final Type keyMetadataType;
 
   private final SparkStructLike wrappedPartition;
+  private final Types.StructType partitionStruct;
   private Row wrapped;
 
+  private static final StructLike EMPTY_PARTITION_INFO = new StructLike() {
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      throw new UnsupportedOperationException("Cannot get a value from an empty partition");
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("Cannot set a value in an empty partition");
+    }
+  };
+
   public SparkDataFile(Types.StructType type, StructType sparkType) {
     this.lowerBoundsType = type.fieldType("lower_bounds");
     this.upperBoundsType = type.fieldType("upper_bounds");
     this.keyMetadataType = type.fieldType("key_metadata");
-    this.wrappedPartition = new SparkStructLike(type.fieldType("partition").asStructType());
+    this.partitionStruct = type.fieldType("partition").asStructType();

Review comment:
       This is necessary for the "Copy" method which is needed for building the BaseScanTasks




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] holdenk commented on pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-709560757


   Oh yeah this is a big one :) I'll try and look at it tomorrow afternoon. Is there a part of it where you would like me to focus?


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510406876



##########
File path: api/src/main/java/org/apache/iceberg/DataFile.java
##########
@@ -80,7 +80,8 @@ static StructType getType(StructType partitionType) {
         UPPER_BOUNDS,
         KEY_METADATA,
         SPLIT_OFFSETS,
-        EQUALITY_IDS
+        EQUALITY_IDS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       Should we document why this is optional @RussellSpitzer or do we prefer to leave the code more pristine and maybe document elsewhere. Definitely something I would imagine forgetting over time.




-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510415243



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java
##########
@@ -41,15 +42,32 @@
 import org.apache.spark.sql.internal.SQLConf;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
 
+@RunWith(Parameterized.class)
 public abstract class TestSparkReaderDeletes extends DeleteReadTests {
 
   private static TestHiveMetastore metastore = null;
   protected static SparkSession spark = null;
   protected static HiveCatalog catalog = null;
 
+  @Parameterized.Parameters(name = "Distributed Planning {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { false },
+        new Object[] { true },
+    };
+  }

Review comment:
       Is there no way to collapse this into a single `Object[]`?
   
   Nevermind. I recall this from some of the other spark tests.




-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499920578



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);

Review comment:
       One awkward thing I've just hit as I've just about finished this refactor, If we make the decision to distribute when we are building the scan and not when we call planFiles, we can't fallback from distributed to not distributed if it doesn't work, unless we maintain two scan objects.
   
   This also makes it hard for us to potentially try both algorithms and use whichever finishes first without building two scan objects :/




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499788561



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       ` private final SparkStructLike wrappedPartition;`




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r492891197



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -32,7 +33,7 @@
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.types.StructType;
 
-public class SparkDataFile implements DataFile {
+public class SparkDataFile implements DataFile, StructLike, Serializable {

Review comment:
       I think we won't have to make it serializable or implement `StructLike` if we use `SparkDataFile` as a wrapper.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483713331



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));
+
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    //Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = new 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);

Review comment:
       Then we could also build those cache's remotely as well, I dunno just spitballing on that. I think if we can avoid registering more serializers it would be best ...




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-709426101


   @holdenk This is the last of the bigger PR's I currently have in flight if you wanted to take a peak. I would be grateful for the review :)


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r485984545



##########
File path: core/src/main/java/org/apache/iceberg/DataFilesTable.java
##########
@@ -57,10 +60,14 @@ public TableScan newScan() {
 
   @Override
   public Schema schema() {
-    Schema schema = new Schema(DataFile.getType(table.spec().partitionType()).fields());
+    List<Types.NestedField> fileFields = Lists.newArrayList();
+    fileFields.addAll(DataFile.getType(table.spec().partitionType()).fields());
+    fileFields.add(ManifestFile.SPEC_ID);
+
+    Schema schema = new Schema(fileFields);
     if (table.spec().fields().size() < 1) {
       // avoid returning an empty struct, which is not always supported. instead, drop the partition field (id 102)
-      return TypeUtil.selectNot(schema, Sets.newHashSet(102));
+      return TypeUtil.selectNot(schema, Sets.newHashSet(DataFile.PARTITION_ID));

Review comment:
       This is a good change, but we should put it in a separate PR.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499778525



##########
File path: core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java
##########
@@ -106,7 +106,7 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche
     return new IncrementalDataTableScan(ops, table, schema, context);
   }
 
-  private static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {

Review comment:
       Should we move this to `SnapshotUtil` then?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499686454



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -100,6 +100,10 @@ public PartitionData copy() {
           found = true;
           fromProjectionPos[i] = j;
         }
+        if (fields.get(i).fieldId() == ManifestFile.SPEC_ID.fieldId()) {

Review comment:
       These modifications allow BaseFile to translate into a SparkRow with the specID as a column




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483701405



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));
+
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    //Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = new 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();

Review comment:
       If we go this route, we would need to add metrics evaluators as well.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499790102



##########
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();

Review comment:
       ah yeah, this is me refactoring things which use to be class fields to method variables and not changing the attributes




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499689856



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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

Review comment:
       We could avoid making BaseTableScan public by swapping both Options and tableOps (and a few other options) to fields that get passed through ... Maybe




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483708676



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));

Review comment:
       I've been thinking about this more and I think combining all of our logic into a single mapper makes 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.

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



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


[GitHub] [iceberg] RussellSpitzer closed pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer closed pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499697678



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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("Distributed 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))
+        .filter(manifestEntries.col("status").notEqual(2)); // not deleted
+
+    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()));
+    }
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    // 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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache =
+        Caffeine.newBuilder().build((CacheLoader<Integer, SpecCacheEntry> & Serializable) specId -> {
+          PartitionSpec spec = broadcastSpecsById.getValue().get(specId);
+          Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : scanFilter;
+          return new SpecCacheEntry(SchemaParser.toJson(spec.schema()), PartitionSpecParser.toJson(spec),
+              ResidualEvaluator.of(spec, filter, isCaseSensitive));
+        });
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {

Review comment:
       We aren't building tasks remotely because DeleteFileIndex and DeleteFileBuilder are both setup in ways that make serialization difficult. DeleteFileIndex uses thread locals and deletefilebuilder owns a ExecutionService. We can probably parallelize one of these in the future though and move all task construction remotely as well. I'm not sure there is a huge benefit to this though.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499709022



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Required for the SparkDataFile SerDe




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499794237



##########
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);

Review comment:
       I think this needs to set `snapshot-id` or `as-of-timestamp` if either one was set in the scan. Otherwise, this just uses the latest table state.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r485998953



##########
File path: core/src/main/java/org/apache/iceberg/DataFilesTable.java
##########
@@ -120,7 +127,8 @@ protected long targetSplitSize(TableOperations ops) {
       super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals);
       this.io = io;
       this.manifest = manifest;
-      this.schema = schema;
+      //Ignore SpecID when reading

Review comment:
       Sorry this is WIP I was mostly getting this out as a "Does this idea even make sense" sort of thing. I wanted to demonstrate in code what we were thinking




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499677524



##########
File path: core/src/main/java/org/apache/iceberg/BaseTableScan.java
##########
@@ -45,7 +45,7 @@
  * Base class for {@link TableScan} implementations.
  */
 @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
-abstract class BaseTableScan implements TableScan {
+public abstract class BaseTableScan implements TableScan {

Review comment:
       We currently need this to be public ignorer to extract table operations and other table settings in the PlanScanAction which doesn't live in the same package.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499676618



##########
File path: api/src/main/java/org/apache/iceberg/DataFile.java
##########
@@ -80,7 +80,8 @@ static StructType getType(StructType partitionType) {
         UPPER_BOUNDS,
         KEY_METADATA,
         SPLIT_OFFSETS,
-        EQUALITY_IDS
+        EQUALITY_IDS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       Current Approach to getting Spec ID into metadata tables

##########
File path: api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java
##########
@@ -45,7 +46,7 @@
  * rows and false if the file cannot contain matching rows. Files may be skipped if and only if the
  * return value of {@code eval} is false.
  */
-public class InclusiveMetricsEvaluator {
+public class InclusiveMetricsEvaluator implements Serializable {

Review comment:
       We need this remotely for filtering ManifestEntries




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499676222



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -32,7 +33,7 @@
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.types.StructType;
 
-public class SparkDataFile implements DataFile {
+public class SparkDataFile implements DataFile, StructLike, Serializable {

Review comment:
       remove




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499789454



##########
File path: core/src/main/java/org/apache/iceberg/V1Metadata.java
##########
@@ -213,7 +213,8 @@ static Schema wrapFileSchema(Types.StructType fileSchema) {
         DataFile.LOWER_BOUNDS,
         DataFile.UPPER_BOUNDS,
         DataFile.KEY_METADATA,
-        DataFile.SPLIT_OFFSETS
+        DataFile.SPLIT_OFFSETS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       The issue with this is that we have tests which compare schemas, if the field is not listed here it still gets included from Datafile which makes the comparisons
   "partition_spec_id" vs "partition_spec_id_r502" which fails. So I mostly added it here so the test would pass




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511152003



##########
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.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.ScanTasks;
+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 =

Review comment:
       I get it now, I thought he was referring to in the actual planning that happens below. Added this restriction to the metadata read as well, I'll have to add a test for this too.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510415243



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java
##########
@@ -41,15 +42,32 @@
 import org.apache.spark.sql.internal.SQLConf;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
 
+@RunWith(Parameterized.class)
 public abstract class TestSparkReaderDeletes extends DeleteReadTests {
 
   private static TestHiveMetastore metastore = null;
   protected static SparkSession spark = null;
   protected static HiveCatalog catalog = null;
 
+  @Parameterized.Parameters(name = "Distributed Planning {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] { false },
+        new Object[] { true },
+    };
+  }

Review comment:
       Is there no way to collapse this into a single `Object[]`?
   
   Nevermind. I recall this from some of the other spark tests.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499682330



##########
File path: core/src/main/java/org/apache/iceberg/V1Metadata.java
##########
@@ -350,6 +351,8 @@ public Object get(int pos) {
           return wrapped.keyMetadata();
         case 12:
           return wrapped.splitOffsets();
+        case 13:
+          return specId();

Review comment:
       This code is only exercised because ManifestEntryTable (possibly incorrectly) only uses V1Metdata methods to try to read manifests.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r504984274



##########
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:
       https://github.com/apache/iceberg/issues/1552




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499698588



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -77,19 +98,47 @@ public SparkDataFile(Types.StructType type, StructType sparkType) {
     upperBoundsPosition = positions.get("upper_bounds");
     keyMetadataPosition = positions.get("key_metadata");
     splitOffsetsPosition = positions.get("split_offsets");
+    specIdPosition = positions.get("partition_spec_id");
+  }
+
+  private SparkDataFile(SparkDataFile other) {

Review comment:
       Copy constructor since we need to actually serialize this representation back to Spark when we distributed CombinedScanTasks. This means we actually need a SparkDataFile object for every BaseFileScanTask




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499791267



##########
File path: core/src/main/java/org/apache/iceberg/BaseTableScan.java
##########
@@ -45,7 +45,7 @@
  * Base class for {@link TableScan} implementations.
  */
 @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
-abstract class BaseTableScan implements TableScan {
+public abstract class BaseTableScan implements TableScan {

Review comment:
       TableOps and Options




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499681268



##########
File path: core/src/main/java/org/apache/iceberg/TableScanContext.java
##########
@@ -156,7 +156,7 @@ TableScanContext fromSnapshotId(long id) {
         caseSensitive, colStats, projectedSchema, selectedColumns, options, id, toSnapshotId);
   }
 
-  Long toSnapshotId() {
+  public Long toSnapshotId() {

Review comment:
       Both of these methods are public so we can determine snapshotsWithin in PlanScanAction




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499793174



##########
File path: core/src/main/java/org/apache/iceberg/BaseFileScanTask.java
##########
@@ -29,7 +29,7 @@
 import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
 import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
 
-class BaseFileScanTask implements FileScanTask {
+public class BaseFileScanTask implements FileScanTask {

Review comment:
       Sounds reasonable, let me see if I can change that




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499690176



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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;

Review comment:
       This block is mostly copied directly from outside configuration for setting up the task combiner




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r505835594



##########
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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) new BaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString,
+          cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))
+          .filter(dataFileEntries.col("status").equalTo(1)); // Added files only

Review comment:
       Another one of those in a private class enums so we can't touch it from here without making the internals public




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499901086



##########
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:
       Logically we should be able to do the same thing with the metrics as well, although with a bit more complication. This PR is going to be pretty large so I'd like to start tackling all that along with pushdowns later.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511078734



##########
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.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.ScanTasks;
+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";

Review comment:
       Let's make it as `plan-mode` with values `local` and `distributed` for now. We can add `auto` later once we have heuristics.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510413308



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java
##########
@@ -59,6 +63,17 @@
   );
   private static SparkSession spark = null;
 
+  private final boolean distributedPlanning;
+
+  @Parameterized.Parameters(name = "Distributed Planning = {0}")

Review comment:
       Came here to ask for `name`, was not disappointed 👍 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r512217138



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -411,6 +411,9 @@ private static void mergeIcebergHadoopConfs(
     try {
       result = Lists.newArrayList(Actions.forTable(table).planScan().withContext(scan.tableScanContext()).execute());
     } catch (Exception e) {
+      if (SparkSession.active().conf().get(PlanScanAction.ICEBERG_TEST_PLAN_MODE).equals("true")) {
+        throw e;

Review comment:
       When we run tests we just want to break if distributed planning fails




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511064483



##########
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.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.ScanTasks;
+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);

Review comment:
       Would it make sense to make it INFO and log additional info as we do in the non-distributed mode?
   
   Here is what we have there:
   
   ```
         LOG.info("Scanning table {} snapshot {} created at {} with filter {}", table,
             snapshot.snapshotId(), formatTimestampMillis(snapshot.timestampMillis()),
             context.rowFilter());
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499676222



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -32,7 +33,7 @@
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.types.StructType;
 
-public class SparkDataFile implements DataFile {
+public class SparkDataFile implements DataFile, StructLike, Serializable {

Review comment:
       removed StructLike, had to keep Serializable for ScanTask Serializability 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499921297



##########
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:
       Sounds reasonable, but let's file an issue to make sure we don't forget about it.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499915738



##########
File path: core/src/main/java/org/apache/iceberg/V1Metadata.java
##########
@@ -213,7 +213,8 @@ static Schema wrapFileSchema(Types.StructType fileSchema) {
         DataFile.LOWER_BOUNDS,
         DataFile.UPPER_BOUNDS,
         DataFile.KEY_METADATA,
-        DataFile.SPLIT_OFFSETS
+        DataFile.SPLIT_OFFSETS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       I'll remove this when we switch to the isolated tests for this action, rather than me testing on just all possible tests and making sure nothing breaks




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r483693807



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));
+
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    //Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = new 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);

Review comment:
       nit: won't work with Kryo




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510408804



##########
File path: core/src/main/java/org/apache/iceberg/BaseTableScan.java
##########
@@ -105,6 +109,14 @@ protected abstract TableScan newRefinedScan(
       TableOperations ops, Snapshot snapshot, Expression rowFilter,
       boolean ignoreResiduals, boolean caseSensitive, boolean colStats);
 
+  public Long fromSnapshotId() {
+    return context().fromSnapshotId();
+  }
+
+  public Long toSnapshotId() {
+    return context.toSnapshotId();
+  }

Review comment:
       👍 

##########
File path: api/src/main/java/org/apache/iceberg/DataFile.java
##########
@@ -80,7 +80,8 @@ static StructType getType(StructType partitionType) {
         UPPER_BOUNDS,
         KEY_METADATA,
         SPLIT_OFFSETS,
-        EQUALITY_IDS
+        EQUALITY_IDS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       Should we document why this is optional @RussellSpitzer or do we prefer to leave the code more pristine and maybe document elsewhere. Definitely something I would imagine forgetting over time.

##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -100,6 +100,10 @@ public PartitionData copy() {
           found = true;
           fromProjectionPos[i] = j;
         }
+        if (fields.get(i).fieldId() == ManifestFile.SPEC_ID.fieldId()) {
+          found = true;
+          fromProjectionPos[i] = 14;
+        }

Review comment:
       This is not related to your PR but while we're here: once we find the projected value and `found` is true, do we need to iterate over the rest of the entries?

##########
File path: core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java
##########
@@ -107,4 +112,47 @@ public static boolean ancestorOf(Table table, long snapshotId, long ancestorSnap
 
     return newFiles;
   }
+
+  public static List<Snapshot> snapshotsWithin(Table table, long fromSnapshotId, long toSnapshotId) {

Review comment:
       Can we put a TODO for the doc or potentially mark this protected? I don't want to block your PR because you extracted it from elsewhere where it was already private, but it would be nice to have a way to mark this with `// Consider snapshotIdsBetween` in one way or another. But I don't want to block your PR so if you'll come back to it later then that's ok.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499778730



##########
File path: core/src/main/java/org/apache/iceberg/TableScanContext.java
##########
@@ -156,7 +156,7 @@ TableScanContext fromSnapshotId(long id) {
         caseSensitive, colStats, projectedSchema, selectedColumns, options, id, toSnapshotId);
   }
 
-  Long toSnapshotId() {
+  public Long toSnapshotId() {

Review comment:
       Maybe these should be public parts of the `TableScan` API?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499677524



##########
File path: core/src/main/java/org/apache/iceberg/BaseTableScan.java
##########
@@ -45,7 +45,7 @@
  * Base class for {@link TableScan} implementations.
  */
 @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
-abstract class BaseTableScan implements TableScan {
+public abstract class BaseTableScan implements TableScan {

Review comment:
       We currently need this to be public in order to extract table operations and other table settings in the PlanScanAction which doesn't live in the same package.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499785513



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -46,18 +47,38 @@
   private final int upperBoundsPosition;
   private final int keyMetadataPosition;
   private final int splitOffsetsPosition;
+  private final int specIdPosition;
   private final Type lowerBoundsType;
   private final Type upperBoundsType;
   private final Type keyMetadataType;
 
   private final SparkStructLike wrappedPartition;
+  private final Types.StructType partitionStruct;
   private Row wrapped;
 
+  private static final StructLike EMPTY_PARTITION_INFO = new StructLike() {
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      return null;

Review comment:
       If size is 0, then any call to `get` or `set` is invalid. In that case, I think it is safer to throw an exception 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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499699567



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);
     }
-
     return tasks;
   }
 
+  private List<CombinedScanTask> planScan(TableScan scan) {
+    // TODO Need to only use distributed planner for supported implementations and add some heuristics about when
+    //  to use
+    if (scan instanceof DataTableScan) {
+      return planDistributedScan(scan);
+    } else {
+      return planLocalScan(scan);
+    }
+  }
+
+  private List<CombinedScanTask> planDistributedScan(TableScan scan) {
+    List<CombinedScanTask> result;
+    try {

Review comment:
       I think ideally we attempt to do both a local and distributed planning in parallel and just use whichever returns first but this could be a future improvement.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r509461196



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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;

Review comment:
       I'm a little afraid of comment rot there, I'm hoping that the fact that we run the tests for this code block with both distributing planning on and off, so if a proper test is added for any changes, it should fail tests when distributed planning is on. I think that is probably the best defense we can do since the comment would have to be on a parent of the Scan class... we also wouldn't be able to warn folks overriding the method in other Scan Impls. 
   
   I think the tests are our best chance 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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r510415296



##########
File path: core/src/main/java/org/apache/iceberg/BaseFile.java
##########
@@ -100,6 +100,10 @@ public PartitionData copy() {
           found = true;
           fromProjectionPos[i] = j;
         }
+        if (fields.get(i).fieldId() == ManifestFile.SPEC_ID.fieldId()) {
+          found = true;
+          fromProjectionPos[i] = 14;
+        }

Review comment:
       We don't, but I don't think it's that much of a time sink




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499785132



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Where is `SparkStructLike` serialized?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499675867



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));
+
+
+    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());
+
+    /*
+    Since we aren't filtering entries on manifest read, filter file entries using Partition information
+    In the normal path this happens with the manifest reader's entries code, filtering elements before we
+    get to this stage.
+     */
+
+    //Build cache of partition evaluators
+    ImmutableMap.Builder<Integer, Evaluator> evalMapBuilder = new 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();

Review comment:
       @kbendick I believe @aokolnychyi was referring to the file metrics (column min/max values etc ...) Which I didn't apply here :) 
   
   I have added them back in the new commits so we are properly filtering on metrics as well. As for Visibility in the UI currently the planning job will just appear as another job, I do think we should probably log some stats from the planning stage as well but we can look into that in a future pr.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] kbendick commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r489133607



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));

Review comment:
       +1 to working with Spark native primitives (aka `Dataset<Row>`) and `mapPartitions`. To me, the biggest benefit is that the visualization of the tasks in the DAG in the Spark UI / History Server would be much more identifiable and there's nothing more frustrating to my users than stages that have very little information about them. Though this should not run nearly as long as the ones that people complain about (in the worst case, there's one job that has a step that basically blocks the history server UI and makes it seem like the job has stalled for 2 or 3 days 😅  - but the job sure is resilient and I blame Spark's data source v1 which didn't provide much support UI wise).
   
   I also agree that combining the logic into a single mapper would likely make sense and would greatly reduce SerDe / serializability concerns.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r492891197



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -32,7 +33,7 @@
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.types.StructType;
 
-public class SparkDataFile implements DataFile {
+public class SparkDataFile implements DataFile, StructLike, Serializable {

Review comment:
       I think we won't have to make it serializable or implement `StructLike` if we use `SparkDataFile` as a wrapper.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511110729



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();
+
+    // Build delete index locally
+    DeleteFileIndex deleteFileIndex = buildDeleteFileIndex();
+
+    SparkDataFile container = new SparkDataFile(partitionStruct, dataFileSchema);
+    List<FileScanTask> tasks = scanTaskDataset.collectAsList().stream().map(row -> {
+      Row dataFile = row.getAs("data_file");
+      SparkDataFile file = container.wrap(dataFile);
+      DeleteFile[] deletes =
+          deleteFileIndex.forDataFile(row.getAs("sequence_number"), file);
+      SpecCacheEntry cached = specCache.get(file.specId());
+      return (FileScanTask) ScanTasks
+          .createBaseFileScanTask(file.copy(), deletes, cached.schemaString, cached.specString, cached.residuals);
+    }).collect(Collectors.toList());
+
+    return CloseableIterable.withNoopClose(tasks);
+  }
+
+  private Dataset<Row> handleIncrementalScan(Dataset<Row> dataFileEntries) {
+    if (context.fromSnapshotId() != null) {
+      LOG.debug("Planning incremental scan from {} to {}", context.fromSnapshotId(), context.toSnapshotId());
+      List<Snapshot> snapshots = SnapshotUtil.snapshotsWithin(table, context.fromSnapshotId(), context.toSnapshotId());
+      List<Long> validSnapshotIds = snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList());
+      return dataFileEntries
+          .filter(dataFileEntries.col("snapshot_id").isin(validSnapshotIds.toArray()))
+          .filter(dataFileEntries.col("status").equalTo(1)); // Added files only
+    } else if (context.snapshotId() != null) {
+      LOG.debug("Planning scan at snapshot {}", context.snapshotId());
+      return dataFileEntries

Review comment:
       Let me add another test too then, because our current test coverage is not catching that behavior




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499795838



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
##########
@@ -385,16 +387,40 @@ private static void mergeIcebergHadoopConfs(
         }
       }
 
-      try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) {
-        this.tasks = Lists.newArrayList(tasksIterable);
-      } catch (IOException e) {
-        throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
-      }
+      this.tasks = planScan(scan);

Review comment:
       The awkward thing about this is that `TableScan` is an API based on refinement, so you'd be creating a new action each time, just like we do in `BaseTableScan`. That's not terrible, but it is a bit of a surprise since the other actions use a builder pattern instead.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499788607



##########
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();

Review comment:
       I know there's debate about it theoretically allowing the compiler to make better choices, but we usually don't declare local variables final.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r496950086



##########
File path: core/src/main/java/org/apache/iceberg/DataFilesTable.java
##########
@@ -57,10 +60,14 @@ public TableScan newScan() {
 
   @Override
   public Schema schema() {
-    Schema schema = new Schema(DataFile.getType(table.spec().partitionType()).fields());
+    List<Types.NestedField> fileFields = Lists.newArrayList();
+    fileFields.addAll(DataFile.getType(table.spec().partitionType()).fields());
+    fileFields.add(ManifestFile.SPEC_ID);
+
+    Schema schema = new Schema(fileFields);
     if (table.spec().fields().size() < 1) {
       // avoid returning an empty struct, which is not always supported. instead, drop the partition field (id 102)
-      return TypeUtil.selectNot(schema, Sets.newHashSet(102));
+      return TypeUtil.selectNot(schema, Sets.newHashSet(DataFile.PARTITION_ID));

Review comment:
       https://github.com/apache/iceberg/pull/1528/files 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.

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511091107



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();

Review comment:
       Can we return early if there are no delete files?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rymurr commented on pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-878084196


   > I have to do some major updates so that it works with deletes. I plan on getting back to this soon but if you let me know what you want extracted maybe we can do some smaller prs real fast?
   
   I really only need the visibility modifications and the util classes in `core` and `api`. We will likely follow a similar strategy to how you are doing it in Spark but those changes aren't needed for us.
   
   I suspect our first iteration isn't focused on V2 delete files.


-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499788270



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Same place, SparkDataFile. It declares the wrapped partition as SparkStructLike




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511107292



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Ok, so it is not for distributed planning as such, it is needed after the planning is done since our DataFiles are actually SparkDataFiles under the hood?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511128454



##########
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.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.ScanTasks;
+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
+
+    dataFileEntries = handleIncrementalScan(dataFileEntries);
+
+    // Build up evaluators and filters for Metrics and Partition values
+    Expression scanFilter = context.rowFilter();
+    boolean isCaseSensitive = context.caseSensitive();
+
+    // Build cache of partition evaluators
+    Broadcast<Map<Integer, Evaluator>> broadcastPartitionEvaluators = buildPartitionEvaluators();
+
+    // Build metric evaluators
+    Broadcast<InclusiveMetricsEvaluator> broadcastMetricsEvaluator = jsc.broadcast(
+        new InclusiveMetricsEvaluator(schema, scanFilter, isCaseSensitive));
+
+    // Cache residual information and Partition spec information
+    Types.StructType partitionStruct = DataFile.getType(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()));
+
+    LoadingCache<Integer, SpecCacheEntry> specCache = buildSpecCache();

Review comment:
       Yeah let's prompt inside the DeleteFile Index forFile so we hit in all uses of DeleteIndex, I'll make that in another PR




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499792334



##########
File path: api/src/main/java/org/apache/iceberg/DataFile.java
##########
@@ -80,7 +80,8 @@ static StructType getType(StructType partitionType) {
         UPPER_BOUNDS,
         KEY_METADATA,
         SPLIT_OFFSETS,
-        EQUALITY_IDS
+        EQUALITY_IDS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       This is optional so Avro won't fail when we attempt to read a file which does not contain this field, this makes it so we don't have to modify the reader and can still have a slot for the SpecID. You can check some of my older commits for some other ways I tried to do this which involved a bit more ... grossness. 
   
   https://github.com/apache/iceberg/pull/1421/commits/4f16c8cf86dc808fb1e7015982827649d3e377f8




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499674602



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkPlannerUtil.java
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed 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.spark.source;
+
+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.Iterator;
+import java.util.List;
+import java.util.Map;
+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.FileScanTask;
+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.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.MapFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SparkPlannerUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPlannerUtil.class);
+  private static final Types.StructType EMPTY_STRUCT = Types.StructType.of();
+
+  public static CloseableIterable<FileScanTask> planFiles(SparkSession spark, TableScan scan) {
+    Snapshot snapshot = scan.snapshot();
+    if (snapshot != null) {
+      LOG.info("Distributed 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(spark, scan, snapshot);
+    } else {
+      LOG.info("Scanning empty table {}", scan.table());
+      return CloseableIterable.empty();
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planFilesFor(SparkSession spark, TableScan scan, Snapshot snapshot) {
+    //This is different for different BaseTableScan implementations
+    if (scan instanceof DataTableScan) {
+      return planDataTableScanFiles(spark, (DataTableScan) scan, snapshot);
+    } else {
+      throw new UnsupportedOperationException(String.format("Unable to do distributed planning for %s",
+          scan.getClass()));
+    }
+  }
+
+  //TODO these two functions probably belong in util now
+  private static String metadataTableName(Table table, MetadataTableType type) {
+    return metadataTableName(table, table.toString(), type);
+  }
+
+  private static String metadataTableName(Table table, String tableName, MetadataTableType type) {
+    if (tableName.contains("/")) {
+      return tableName + "#" + type;
+    } else if (tableName.startsWith("hadoop.")) {
+      // for HadoopCatalog tables, use the table location to load the metadata table
+      // because IcebergCatalog uses HiveCatalog when the table is identified by name
+      return table.location() + "#" + type;
+    } else if (tableName.startsWith("hive.")) {
+      // HiveCatalog prepend a logical name which we need to drop for Spark 2.4
+      return tableName.replaceFirst("hive\\.", "") + "." + type;
+    } else {
+      return tableName + "." + type;
+    }
+  }
+
+  private static CloseableIterable<FileScanTask> planDataTableScanFiles(
+      SparkSession spark,
+      DataTableScan scan,
+      Snapshot snapshot) {
+
+    //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(scan.table(), MetadataTableType.FILES);
+    Dataset<Row> dataFileRows =
+        spark.read()
+            .format("iceberg")
+            .load(dataFilesMetadataTable);
+
+    Types.StructType partitionStruct = DataFile.getType(scan.table().spec().partitionType());
+    StructType sparkSchema = dataFileRows.schema();
+
+    Dataset<SparkDataFile> dataFiles =
+        dataFileRows.mapPartitions((Iterator<Row> it) -> {
+          //Wrap is mutable here and not thread safe
+              SparkDataFile container = new SparkDataFile(partitionStruct, sparkSchema);
+              return Streams.stream(it).map(container::wrap).iterator();
+              },
+            Encoders.javaSerialization(SparkDataFile.class));

Review comment:
       While I was able to remove the serialization here, we still need SparkDataFiles to be serializable since we need to send them back out in the ScanTask objects. Please see the updated commits, lots of new changes




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499782251



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java
##########
@@ -19,11 +19,12 @@
 
 package org.apache.iceberg.spark;
 
+import java.io.Serializable;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.types.Types;
 import org.apache.spark.sql.Row;
 
-public class SparkStructLike implements StructLike {
+public class SparkStructLike implements StructLike, Serializable {

Review comment:
       Shouldn't `SparkDataFile` be serialized as an internal row and not using Java serialization?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499777839



##########
File path: core/src/main/java/org/apache/iceberg/BaseTableScan.java
##########
@@ -45,7 +45,7 @@
  * Base class for {@link TableScan} implementations.
  */
 @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
-abstract class BaseTableScan implements TableScan {
+public abstract class BaseTableScan implements TableScan {

Review comment:
       What parts need to be accessed that aren't part of the public API? Should we just add those to the `TableScan` API?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511095858



##########
File path: spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java
##########
@@ -46,18 +47,38 @@
   private final int upperBoundsPosition;
   private final int keyMetadataPosition;
   private final int splitOffsetsPosition;
+  private final int specIdPosition;
   private final Type lowerBoundsType;
   private final Type upperBoundsType;
   private final Type keyMetadataType;
 
   private final SparkStructLike wrappedPartition;
+  private final Types.StructType partitionStruct;
   private Row wrapped;
 
+  private static final StructLike EMPTY_PARTITION_INFO = new StructLike() {
+    @Override
+    public int size() {
+      return 0;
+    }
+
+    @Override
+    public <T> T get(int pos, Class<T> javaClass) {
+      throw new UnsupportedOperationException("Cannot get a value from an empty partition");
+    }
+
+    @Override
+    public <T> void set(int pos, T value) {
+      throw new UnsupportedOperationException("Cannot set a value in an empty partition");
+    }
+  };
+
   public SparkDataFile(Types.StructType type, StructType sparkType) {
     this.lowerBoundsType = type.fieldType("lower_bounds");
     this.upperBoundsType = type.fieldType("upper_bounds");
     this.keyMetadataType = type.fieldType("key_metadata");
-    this.wrappedPartition = new SparkStructLike(type.fieldType("partition").asStructType());
+    this.partitionStruct = type.fieldType("partition").asStructType();

Review comment:
       How do we use `partitionStruct` that was introduced?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499791267



##########
File path: core/src/main/java/org/apache/iceberg/BaseTableScan.java
##########
@@ -45,7 +45,7 @@
  * Base class for {@link TableScan} implementations.
  */
 @SuppressWarnings("checkstyle:OverloadMethodsDeclarationOrder")
-abstract class BaseTableScan implements TableScan {
+public abstract class BaseTableScan implements TableScan {

Review comment:
       TableOps and Options, I'm not sure we want TableOps to be public but Options feels like it should have been public already, currently only "option" is public for setting an option




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r502658763



##########
File path: api/src/main/java/org/apache/iceberg/DataFile.java
##########
@@ -80,7 +80,8 @@ static StructType getType(StructType partitionType) {
         UPPER_BOUNDS,
         KEY_METADATA,
         SPLIT_OFFSETS,
-        EQUALITY_IDS
+        EQUALITY_IDS,
+        ManifestFile.SPEC_ID.asOptional()

Review comment:
       Note to myself @RussellSpitzer : remove this




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #1421: WIP - Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r499695504



##########
File path: spark/src/main/java/org/apache/iceberg/actions/PlanScanAction.java
##########
@@ -0,0 +1,271 @@
+/*
+ * 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("Distributed 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))
+        .filter(manifestEntries.col("status").notEqual(2)); // not deleted

Review comment:
       These numbers are both magic because ManifestEntry is protected and we can't access the constants from here, may want to change this as well




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#issuecomment-877876500


   I have to do some major updates so that it works with deletes. I plan on getting back to this soon but if you let me know what you want extracted maybe we can do some smaller prs real fast?


-- 
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-unsubscribe@iceberg.apache.org

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #1421: Add a Parallelized Spark Job Planning Path

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #1421:
URL: https://github.com/apache/iceberg/pull/1421#discussion_r511065515



##########
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.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.ScanTasks;
+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;

Review comment:
       super nit: we call it `sparkContext` in all other actions, shall we make it consistent?




----------------------------------------------------------------
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:
users@infra.apache.org



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