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/05 15:13:50 UTC

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

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