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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
