From dee06f65c093d938cde24d5ee53fc81c65bc927b Mon Sep 17 00:00:00 2001 From: yaohua Date: Thu, 11 Nov 2021 16:41:46 -0800 Subject: [PATCH 01/21] first draft --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../expressions/namedExpressions.scala | 67 +++ .../catalyst/plans/logical/LogicalPlan.scala | 7 + .../datasources/v2/DataSourceV2Relation.scala | 4 +- .../sql/execution/DataSourceScanExec.scala | 31 +- .../execution/datasources/FileFormat.scala | 11 +- .../execution/datasources/FileScanRDD.scala | 114 ++++- .../datasources/FileSourceStrategy.scala | 4 +- .../datasources/LogicalRelation.scala | 25 +- .../datasources/FileMetadataStructSuite.scala | 429 ++++++++++++++++++ 10 files changed, 677 insertions(+), 17 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 068886e5c961..53d60a51f00c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -965,7 +965,7 @@ class Analyzer(override val catalogManager: CatalogManager) } private def addMetadataCol(plan: LogicalPlan): LogicalPlan = plan match { - case r: DataSourceV2Relation => r.withMetadataColumns() + case s: ExposesMetadataColumns => s.withMetadataColumns() case p: Project => p.copy( projectList = p.metadataOutput ++ p.projectList, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 71f193e51074..40f40f028665 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -438,3 +438,70 @@ object VirtualColumn { val groupingIdName: String = "spark_grouping_id" val groupingIdAttribute: UnresolvedAttribute = UnresolvedAttribute(groupingIdName) } + +/** + * The internal representation of the hidden metadata column + */ +class MetadataAttribute( + override val name: String, + override val dataType: DataType, + override val nullable: Boolean = true, + override val metadata: Metadata = Metadata.empty)( + override val exprId: ExprId = NamedExpression.newExprId, + override val qualifier: Seq[String] = Seq.empty[String]) + extends AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier) { + + // use to resolve supported metadata column references (e.g. different casings) + override def withName(newName: String): MetadataAttribute = { + if (name == newName) { + this + } else { + MetadataAttribute(newName, dataType, nullable, metadata)(exprId, qualifier) + } + } + + override def withNullability(newNullability: Boolean): MetadataAttribute = { + if (nullable == newNullability) { + this + } else { + MetadataAttribute(name, dataType, newNullability, metadata)(exprId, qualifier) + } + } + + override def withQualifier(newQualifier: Seq[String]): MetadataAttribute = { + if (qualifier == newQualifier) { + this + } else { + MetadataAttribute(name, dataType, nullable, metadata)(exprId, newQualifier) + } + } + + override def withExprId(newExprId: ExprId): MetadataAttribute = { + if (exprId == newExprId) { + this + } else { + MetadataAttribute(name, dataType, nullable, metadata)(newExprId, qualifier) + } + } + + override def withDataType(newType: DataType): MetadataAttribute = { + MetadataAttribute(name, newType, nullable, metadata)(exprId, qualifier) + } + + override def newInstance(): MetadataAttribute = + MetadataAttribute(name, dataType, nullable, metadata)(exprId, qualifier) + + override def withMetadata(newMetadata: Metadata): MetadataAttribute = { + MetadataAttribute(name, dataType, nullable, newMetadata)(exprId, qualifier) + } +} + +object MetadataAttribute { + + def apply(name: String, dataType: DataType): MetadataAttribute = + new MetadataAttribute(name, dataType, true)() + + def apply(name: String, dataType: DataType, nullable: Boolean, metadata: Metadata) + (exprId: ExprId, qualifier: Seq[String]): MetadataAttribute = + new MetadataAttribute(name, dataType, nullable, metadata)(exprId, qualifier) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 7c31a0091811..66ef9650b5ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -276,3 +276,10 @@ object LogicalPlanIntegrity { checkIfSameExprIdNotReused(plan) && hasUniqueExprIdsForOutput(plan) } } + +/** + * A logical plan node with exposed metadata columns + */ +trait ExposesMetadataColumns extends LogicalPlan { + def withMetadataColumns(): ExposesMetadataColumns +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index eff0cd8364a8..d93864991fc3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.{ExposesMetadataColumns, LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.{truncatedString, CharVarcharUtils} import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier, MetadataColumn, SupportsMetadataColumns, Table, TableCapability} import org.apache.spark.sql.connector.read.{Scan, Statistics => V2Statistics, SupportsReportStatistics} @@ -44,7 +44,7 @@ case class DataSourceV2Relation( catalog: Option[CatalogPlugin], identifier: Option[Identifier], options: CaseInsensitiveStringMap) - extends LeafNode with MultiInstanceRelation with NamedRelation { + extends LeafNode with MultiInstanceRelation with NamedRelation with ExposesMetadataColumns { import DataSourceV2Implicits._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index a3e3a2d54e32..81dbb89c2b1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.datasources.v2.PushedDownOperators import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.StructType @@ -194,10 +195,17 @@ case class FileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { + lazy val outputMetadataStruct: Option[MetadataAttribute] = + output.collectFirst { case meta: MetadataAttribute => meta } + // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + // schema without file metadata column + val fileSchema = if (outputMetadataStruct.isEmpty) schema else { + StructType.fromAttributes(output.filterNot(_.isInstanceOf[MetadataAttribute])) + } + relation.fileFormat.supportBatch(relation.sparkSession, fileSchema) } private lazy val needsUnsafeRowConversion: Boolean = { @@ -212,7 +220,16 @@ case class FileSourceScanExec( relation.fileFormat.vectorTypes( requiredSchema = requiredSchema, partitionSchema = relation.partitionSchema, - relation.sparkSession.sessionState.conf) + relation.sparkSession.sessionState.conf).map { vectorTypes => + val metadataVectorClz = + if (relation.sparkSession.sessionState.conf.offHeapColumnVectorEnabled) { + classOf[OffHeapColumnVector].getName + } else { + classOf[OnHeapColumnVector].getName + } + // for column-based file format, append metadata columns' vector type classes if any + vectorTypes ++ (if (outputMetadataStruct.isDefined) Seq(metadataVectorClz) else Seq.empty) + } private lazy val driverMetrics: HashMap[String, Long] = HashMap.empty @@ -355,7 +372,9 @@ case class FileSourceScanExec( @transient private lazy val pushedDownFilters = { val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) - dataFilters.flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) + dataFilters + .filterNot(_.references.exists(_.isInstanceOf[MetadataAttribute])) + .flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) } override lazy val metadata: Map[String, String] = { @@ -597,7 +616,8 @@ case class FileSourceScanExec( } } - new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions) + new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions, + requiredSchema, outputMetadataStruct) } /** @@ -653,7 +673,8 @@ case class FileSourceScanExec( val partitions = FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) - new FileScanRDD(fsRelation.sparkSession, readFile, partitions) + new FileScanRDD(fsRelation.sparkSession, readFile, partitions, + requiredSchema, outputMetadataStruct) } // Filters unused DynamicPruningExpression expressions - one which has been replaced diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index beb1f4d38e8b..7dfb9168b275 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} /** @@ -171,6 +171,15 @@ trait FileFormat { def supportFieldName(name: String): Boolean = true } +object FileFormat { + + // supported metadata columns for hadoop fs relation + val FILE_METADATA_COLUMNS: MetadataAttribute = MetadataAttribute("_metadata", + new StructType() + .add(StructField("file_path", StringType)) + .add(StructField("file_name", StringType))) +} + /** * The base class file format that is based on text file. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index d817249005f0..fcbf3bdbc318 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -17,17 +17,23 @@ package org.apache.spark.sql.execution.datasources -import java.io.{Closeable, FileNotFoundException, IOException} +import java.io.{Closeable, File, FileNotFoundException, IOException} import scala.util.control.NonFatal import org.apache.spark.{Partition => RDDPartition, SparkUpgradeException, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions.{GenericRow, MetadataAttribute, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources.FileFormat._ +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.NextIterator /** @@ -57,11 +63,15 @@ case class PartitionedFile( class FileScanRDD( @transient private val sparkSession: SparkSession, readFunction: (PartitionedFile) => Iterator[InternalRow], - @transient val filePartitions: Seq[FilePartition]) + @transient val filePartitions: Seq[FilePartition], + val requiredSchema: StructType = StructType(Seq.empty), + val metadataStruct: Option[MetadataAttribute] = None) extends RDD[InternalRow](sparkSession.sparkContext, Nil) { private val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles private val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles + private val offHeapColumnVectorEnabled = + sparkSession.sessionState.conf.offHeapColumnVectorEnabled override def compute(split: RDDPartition, context: TaskContext): Iterator[InternalRow] = { val iterator = new Iterator[Object] with AutoCloseable { @@ -103,6 +113,98 @@ class FileScanRDD( context.killTaskIfInterrupted() (currentIterator != null && currentIterator.hasNext) || nextIterator() } + + /////////////////////////// + // FILE METADATA METHODS // + /////////////////////////// + + // use to join with an unsafe row, will only be updated when the current file is changed + @volatile var metadataStructUnsafeRow: UnsafeRow = _ + // use to append to an internal row, will only be updated when the current file is changed + @volatile var metadataStructGenericRow: Row = _ + + // Create a off/on heap WritableColumnVector + private def createColumnVector(numRows: Int, dataType: DataType): WritableColumnVector = { + if (offHeapColumnVectorEnabled) { + new OffHeapColumnVector(numRows, dataType) + } else { + new OnHeapColumnVector(numRows, dataType) + } + } + + /** + * For each partitioned file, metadata columns for each record in the file are exactly same. + * Only update metadata columns when `currentFile` is changed. + */ + private def updateMetadataStruct(): Unit = + metadataStruct.foreach { + case meta if FILE_METADATA_COLUMNS.sameRef(meta) => + if (currentFile == null) { + metadataStructUnsafeRow = new UnsafeRow(1) + metadataStructGenericRow = new GenericRow(1) + } else { + // make an generic row + metadataStructGenericRow = Row.fromSeq(Seq( + // file_path + UTF8String.fromString(new File(currentFile.filePath).toString), + // file_name + UTF8String.fromString(currentFile.filePath.split("/").last))) + + // convert the generic row to an unsafe row + val unsafeRowConverter = { + val converter = UnsafeProjection.create( + Array(FILE_METADATA_COLUMNS.dataType)) + (row: Row) => { + converter(CatalystTypeConverters.convertToCatalyst(row) + .asInstanceOf[InternalRow]) + } + } + metadataStructUnsafeRow = + unsafeRowConverter(Row.fromSeq(Seq(metadataStructGenericRow))) + } + } + + /** + * Create a writable column vector containing all required metadata fields + */ + private def createMetadataStructColumnVector(c: ColumnarBatch): WritableColumnVector = { + val columnVector = createColumnVector(c.numRows(), FILE_METADATA_COLUMNS.dataType) + val filePathBytes = new File(currentFile.filePath).toString.getBytes + val fileNameBytes = currentFile.filePath.split("/").last.getBytes + + var rowId = 0 + // use a tight-loop for better performance + while (rowId < c.numRows()) { + // _file_path + columnVector.getChild(0).putByteArray(rowId, filePathBytes) + // _file_name + columnVector.getChild(1).putByteArray(rowId, fileNameBytes) + rowId += 1 + } + columnVector + } + + /** + * Add metadata struct at the end of nextElement if needed. + * For different row implementations, use different methods to update and append. + */ + private def addMetadataStructIfNeeded(nextElement: Object): Object = + metadataStruct.map { + case meta if FILE_METADATA_COLUMNS.sameRef(meta) => + nextElement match { + case c: ColumnarBatch => + val columnVectorArr = Array.tabulate(c.numCols())(c.column) ++ + Array(createMetadataStructColumnVector(c)) + new ColumnarBatch(columnVectorArr, c.numRows()) + case u: UnsafeRow => + val joiner = + GenerateUnsafeRowJoiner.create(requiredSchema, Seq(meta).toStructType) + joiner.join(u, metadataStructUnsafeRow) + case i: InternalRow => + InternalRow.fromSeq(i.toSeq(requiredSchema) ++ metadataStructGenericRow.toSeq) + } + }.getOrElse(nextElement) + def next(): Object = { val nextElement = currentIterator.next() // TODO: we should have a better separation of row based and batch based scan, so that we @@ -119,7 +221,7 @@ class FileScanRDD( } inputMetrics.incRecordsRead(1) } - nextElement + addMetadataStructIfNeeded(nextElement) } private def readCurrentFile(): Iterator[InternalRow] = { @@ -135,6 +237,7 @@ class FileScanRDD( private def nextIterator(): Boolean = { if (files.hasNext) { currentFile = files.next() + updateMetadataStruct() logInfo(s"Reading File $currentFile") // Sets InputFileBlockHolder for the file block's information InputFileBlockHolder.set(currentFile.filePath, currentFile.start, currentFile.length) @@ -202,6 +305,7 @@ class FileScanRDD( } } else { currentFile = null + updateMetadataStruct() InputFileBlockHolder.unset() false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 1bfde7515dc9..75efb432e94b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -212,7 +212,9 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { val outputSchema = readDataColumns.toStructType logInfo(s"Output Data Schema: ${outputSchema.simpleString(5)}") - val outputAttributes = readDataColumns ++ partitionColumns + // outputAttributes should also include referenced metadata columns at the every end + val outputAttributes = readDataColumns ++ partitionColumns ++ + plan.references.collect { case meta: MetadataAttribute => meta } val scan = FileSourceScanExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index cfe5f046bd7b..0c558580dc09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -20,7 +20,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.{ExposesMetadataColumns, LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.{truncatedString, CharVarcharUtils} import org.apache.spark.sql.sources.BaseRelation @@ -32,7 +32,7 @@ case class LogicalRelation( output: Seq[AttributeReference], catalogTable: Option[CatalogTable], override val isStreaming: Boolean) - extends LeafNode with MultiInstanceRelation { + extends LeafNode with MultiInstanceRelation with ExposesMetadataColumns { // Only care about relation when canonicalizing. override def doCanonicalize(): LogicalPlan = copy( @@ -67,6 +67,27 @@ case class LogicalRelation( s"Relation ${catalogTable.map(_.identifier.unquotedString).getOrElse("")}" + s"[${truncatedString(output, ",", maxFields)}] $relation" } + + override lazy val metadataOutput: Seq[AttributeReference] = relation match { + case _: HadoopFsRelation => + val resolve = conf.resolver + val outputNames = outputSet.map(_.name) + def isOutputColumn(col: AttributeReference): Boolean = { + outputNames.exists(name => resolve(col.name, name)) + } + // filter out metadata columns that have names conflicting with output columns. if the file + // has a column "_metadata", then the data column should be returned not the metadata column + Seq(FileFormat.FILE_METADATA_COLUMNS).filterNot(isOutputColumn) + case _ => Nil + } + + override def withMetadataColumns(): LogicalRelation = { + if (metadataOutput.nonEmpty) { + this.copy(output = output ++ metadataOutput) + } else { + this + } + } } object LogicalRelation { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala new file mode 100644 index 000000000000..a8c34ef939de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala @@ -0,0 +1,429 @@ +/* + * 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.spark.sql.execution.datasources + +import java.io.File +import java.nio.file.Files + +import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} + +class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { + + val data0: String = + """ + |jack,24,12345,uom + |""".stripMargin + + val data1: String = + """ + |lily,31,,ucb + |""".stripMargin + + val schema: StructType = new StructType() + .add(StructField("name", StringType)) + .add(StructField("age", IntegerType)) + .add(StructField("id", LongType)) + .add(StructField("university", StringType)) + + val schemaWithNameConflicts: StructType = new StructType() + .add(StructField("name", StringType)) + .add(StructField("age", IntegerType)) + .add(StructField("id", LongType)) + .add(StructField("_metadata.FILE_NAME", StringType)) + + private val METADATA_FILE_PATH = "_metadata.file_path" + + private val METADATA_FILE_NAME = "_metadata.file_name" + + /** + * Create a CSV file named `fileName` with `data` under `dir` directory. + */ + private def createCSVFile(data: String, dir: File, fileName: String): String = { + val dataFile = new File(dir, s"/$fileName") + dataFile.getParentFile.mkdirs() + val bytes = data.getBytes() + Files.write(dataFile.toPath, bytes) + dataFile.getCanonicalPath + } + + /** + * This test wrapper will test for both row-based and column-based file formats (csv and parquet) + * 1. read data0 and data1 and write them as testFileFormat: f0 and f1 + * 2. read both f0 and f1, return the df to the downstream for further testing + * 3. construct actual metadata map for both f0 and f1 to the downstream for further testing + * + * The final df will have data: + * jack | 24 | 12345 | uom + * lily | 31 | null | ucb + * + * The schema of the df will be the `fileSchema` provided to this method + * + * This test wrapper will provide a `df` and actual metadata map `f0`, `f1` + */ + private def metadataColumnsTest( + testName: String, fileSchema: StructType) + (f: (DataFrame, Map[String, Any], Map[String, Any]) => Unit): Unit = { + Seq("csv", "parquet").foreach { testFileFormat => + test(s"metadata columns ($testFileFormat): " + testName) { + withTempDir { dir => + // read data0 as CSV and write f0 as testFileFormat + val df0 = spark.read.schema(fileSchema).csv( + createCSVFile(data0, dir, "temp/0") + ) + val f0Path = new File(dir, "data/f0").getCanonicalPath + df0.coalesce(1).write.format(testFileFormat).save(f0Path) + + // read data1 as CSV and write f1 as testFileFormat + val df1 = spark.read.schema(fileSchema).csv( + createCSVFile(data1, dir, "temp/1") + ) + val f1Path = new File(dir, "data/f1").getCanonicalPath + df1.coalesce(1).write.format(testFileFormat).save(f1Path) + + // read both f0 and f1 + val df = spark.read.format(testFileFormat).schema(fileSchema) + .load(new File(dir, "data").getCanonicalPath + "/*") + + val realF0 = new File(dir, "data/f0").listFiles() + .filter(_.getName.endsWith(s".$testFileFormat")).head + val realF1 = new File(dir, "data/f1").listFiles() + .filter(_.getName.endsWith(s".$testFileFormat")).head + + // construct f0 and f1 metadata data + val f0Metadata = Map( + METADATA_FILE_PATH -> realF0.toURI.toString, + METADATA_FILE_NAME -> realF0.getName + ) + val f1Metadata = Map( + METADATA_FILE_PATH -> realF1.toURI.toString, + METADATA_FILE_NAME -> realF1.getName + ) + + f(df, f0Metadata, f1Metadata) + } + } + } + } + + metadataColumnsTest("read partial/all metadata columns", schema) { (df, f0, f1) => + // read all available metadata columns + checkAnswer( + df.select("name", "age", "id", "university", + METADATA_FILE_NAME, METADATA_FILE_PATH), + Seq( + Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), + Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + ) + ) + + // read a part of metadata columns + checkAnswer( + df.select("name", "university", METADATA_FILE_NAME), + Seq( + Row("jack", "uom", f0(METADATA_FILE_NAME)), + Row("lily", "ucb", f1(METADATA_FILE_NAME)) + ) + ) + } + + metadataColumnsTest("read metadata columns with random ordering", schema) { (df, f0, f1) => + // read a part of metadata columns with random ordering + checkAnswer( + df.select(METADATA_FILE_NAME, "name", METADATA_FILE_PATH, "university"), + Seq( + Row(f0(METADATA_FILE_NAME), "jack", f0(METADATA_FILE_PATH), "uom"), + Row(f1(METADATA_FILE_NAME), "lily", f1(METADATA_FILE_PATH), "ucb") + ) + ) + } + + metadataColumnsTest("read metadata columns with expressions", schema) { (df, f0, f1) => + checkAnswer( + df.select( + // substring of file name + substring(col(METADATA_FILE_NAME), 1, 3), + // get the file format + substring_index(col(METADATA_FILE_PATH), ".", -1).as("_file_format") + ), + Seq( + Row( + f0(METADATA_FILE_NAME).toString.substring(0, 3), // sql substring vs scala substring + f0(METADATA_FILE_PATH).toString.split("\\.").takeRight(1).head + ), + Row( + f1(METADATA_FILE_NAME).toString.substring(0, 3), // sql substring vs scala substring + f1(METADATA_FILE_PATH).toString.split("\\.").takeRight(1).head + ) + ) + ) + } + + metadataColumnsTest("select all will not select metadata columns", schema) { (df, _, _) => + checkAnswer( + df.select("*"), + Seq( + Row("jack", 24, 12345L, "uom"), + Row("lily", 31, null, "ucb") + ) + ) + } + + metadataColumnsTest("metadata columns is struct, user data column is string", + schemaWithNameConflicts) { (df, f0, f1) => + // here: the data has the schema: name, age, id, _metadata.file_name + checkAnswer( + df.select("name", "age", "id", "`_metadata.FILE_NAME`", + METADATA_FILE_NAME, METADATA_FILE_PATH), + Seq( + Row("jack", 24, 12345L, "uom", + // user data will not be overwritten, + // and we still can read metadata columns correctly + f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), + Row("lily", 31, null, "ucb", + // user data will not be overwritten, + // and we still can read metadata columns correctly + f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + ) + ) + } + + metadataColumnsTest("select only metadata columns", schema) { (df, f0, f1) => + checkAnswer( + df.select(METADATA_FILE_NAME, METADATA_FILE_PATH), + Seq( + Row(f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), + Row(f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + ) + ) + } + + metadataColumnsTest("select and re-select", schema) { (df, f0, f1) => + // test and make sure we are not accidentally making unsafe row + // to the more general internal row, thus it will fail to re-select + checkAnswer( + df.select("name", "age", "id", "university", + METADATA_FILE_NAME, METADATA_FILE_PATH) + .select("name", "file_path"), // cast _metadata.file_path as file_path + Seq( + Row("jack", f0(METADATA_FILE_PATH)), + Row("lily", f1(METADATA_FILE_PATH)) + ) + ) + } + + metadataColumnsTest("alias", schema) { (df, f0, f1) => + + val aliasDF = df.select( + Column("name").as("myName"), + Column("age").as("myAge"), + Column(METADATA_FILE_NAME).as("myFileName") + ) + + // check schema + val expectedSchema = new StructType() + .add(StructField("myName", StringType)) + .add(StructField("myAge", IntegerType)) + .add(StructField("myFileName", StringType)) + + assert(aliasDF.schema.fields.toSet == expectedSchema.fields.toSet) + + // check data + checkAnswer( + aliasDF, + Seq( + Row("jack", 24, f0(METADATA_FILE_NAME)), + Row("lily", 31, f1(METADATA_FILE_NAME)) + ) + ) + } + + metadataColumnsTest("filter", schema) { (df, f0, _) => + checkAnswer( + df.select("name", "age", METADATA_FILE_NAME) + .where(Column(METADATA_FILE_NAME) === f0(METADATA_FILE_NAME)), + Seq( + // _file_name == f0's name, so we will only have 1 row + Row("jack", 24, f0(METADATA_FILE_NAME)) + ) + ) + } + + Seq(true, false).foreach { caseSensitive => + metadataColumnsTest(s"upper/lower case when case " + + s"sensitive is $caseSensitive", schemaWithNameConflicts) { (df, f0, f1) => + withSQLConf("spark.sql.caseSensitive" -> caseSensitive.toString) { + + // file schema: name, age, id, _metadata._FILE_NAME + if (caseSensitive) { + // for case sensitive mode: + // _METADATA.FILE_PATH is not a part of user schema or metadata columns + val ex = intercept[Exception] { + df.select("name", "age", "_METADATA.FILE_PATH").show() + } + assert(ex.getMessage.contains("_METADATA.FILE_PATH")) + + // for case sensitive mode: + // `_metadata.FILE_NAME` is in the user schema + // _metadata.file_name is metadata columns + checkAnswer( + df.select("name", "age", "id", "`_metadata.FILE_NAME`", + "_metadata.file_name"), + Seq( + Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME)), + Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME)) + ) + ) + } else { + // for case insensitive mode: + // `_metadata.file_name`, `_metadata.FILE_NAME` are all from the user schema. + // different casings of _metadata.file_path is metadata columns + checkAnswer( + df.select("name", "age", "id", + "`_metadata.file_name`", "`_metadata.FILE_NAME`", + // metadata columns + "_metadata.file_path", "_metadata.FILE_PATH"), + Seq( + Row("jack", 24, 12345L, "uom", "uom", + f0(METADATA_FILE_PATH), f0(METADATA_FILE_PATH)), + Row("lily", 31, null, "ucb", "ucb", + f1(METADATA_FILE_PATH), f1(METADATA_FILE_PATH)) + ) + ) + } + } + } + } + + Seq("true", "false").foreach { offHeapColumnVectorEnabled => + withSQLConf("spark.sql.columnVector.offheap.enabled" -> offHeapColumnVectorEnabled) { + metadataColumnsTest(s"read metadata with " + + s"offheap set to $offHeapColumnVectorEnabled", schema) { (df, f0, f1) => + // read all available metadata columns + checkAnswer( + df.select("name", "age", "id", "university", + METADATA_FILE_NAME, METADATA_FILE_PATH), + Seq( + Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), + Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + ) + ) + + // read a part of metadata columns + checkAnswer( + df.select("name", "university", METADATA_FILE_NAME), + Seq( + Row("jack", "uom", f0(METADATA_FILE_NAME)), + Row("lily", "ucb", f1(METADATA_FILE_NAME)) + ) + ) + } + } + } + + ////////////////////////// + // TEST METADATA STRUCT // + ////////////////////////// + + // has _metadata.file_name + val jsonData0 = + """ + |{ + | "name":"jack", + | "_metadata":{ + | "age":24, + | "file_name":"jack.json" + | } + |} + |""".stripMargin + val jsonSchema0: StructType = new StructType() + .add(StructField("name", StringType)) + .add(StructField("_metadata", new StructType() + .add(StructField("age", IntegerType)) + .add(StructField("file_name", StringType)))) + + // no naming conflicts at all + val jsonData1 = + """ + |{ + | "name":"jack", + | "metadata":{ + | "age":24, + | "file_name":"jack.json" + | } + |} + |""".stripMargin + val jsonSchema1: StructType = new StructType() + .add(StructField("name", StringType)) + .add(StructField("metadata", new StructType() + .add(StructField("age", IntegerType)) + .add(StructField("file_name", StringType)))) + + /** + * Create a JSON file named `fileName` with `data` under `dir` directory. + */ + private def createJSONFile(data: String, dir: File, fileName: String): String = { + val dataFile = new File(dir, s"/$fileName") + dataFile.getParentFile.mkdirs() + val bytes = data.filter(_ >= ' ').getBytes + Files.write(dataFile.toPath, bytes) + dataFile.getCanonicalPath + } + + test("test data schema has _metadata struct") { + withTempDir { dir => + + // 0 - select metadata will fail when analysis + val df0 = spark.read.schema(jsonSchema0).json( + createJSONFile(jsonData0, dir, "temp/0")) + checkAnswer( + df0.select("name", "_metadata.file_name"), + Row("jack", "jack.json") + ) + val ex0 = intercept[AnalysisException] { + df0.select("name", "_metadata.file_path").show() + } + assert(ex0.getMessage.contains("No such struct field file_path in age, file_name")) + + // 1 - no conflict, everything is fine + val df1 = spark.read.schema(jsonSchema1).json( + createJSONFile(jsonData1, dir, "temp/1")) + + // get metadata + val f1 = new File(dir, "temp/1") + val metadata = Map( + METADATA_FILE_PATH -> f1.toURI.toString, + METADATA_FILE_NAME -> f1.getName + ) + + checkAnswer( + df1.select("name", "metadata.file_name", + "_metadata.file_path", "_metadata.file_name", + "_metadata"), + Row("jack", "jack.json", + metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME), + // struct of _metadata + Row( + metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME)) + ) + ) + } + } +} From 06ac79ef9a0a734b29ca79fe5e454ae1b555d3c6 Mon Sep 17 00:00:00 2001 From: yaohua Date: Thu, 11 Nov 2021 16:44:57 -0800 Subject: [PATCH 02/21] nit of renaming file --- ...leMetadataStructSuite.scala => FileMetadataColumnsSuite.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/{FileMetadataStructSuite.scala => FileMetadataColumnsSuite.scala} (100%) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala similarity index 100% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala From fc043fd5e2fa65aadae7648ef67851ffa6d379c9 Mon Sep 17 00:00:00 2001 From: yaohua Date: Fri, 12 Nov 2021 13:28:55 -0800 Subject: [PATCH 03/21] more meta --- .../sql/execution/PartitionedFileUtil.scala | 6 +- .../execution/datasources/FileFormat.scala | 16 +- .../execution/datasources/FileScanRDD.scala | 58 ++++-- .../FileMetadataColumnsSuite.scala | 184 +++++++++++------- 4 files changed, 176 insertions(+), 88 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala index 7dece29eb021..4cccd4132e91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -36,7 +36,8 @@ object PartitionedFileUtil { val remaining = file.getLen - offset val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining val hosts = getBlockHosts(getBlockLocations(file), offset, size) - PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts) + PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts, + file.getModificationTime, file.getLen) } } else { Seq(getPartitionedFile(file, filePath, partitionValues)) @@ -48,7 +49,8 @@ object PartitionedFileUtil { filePath: Path, partitionValues: InternalRow): PartitionedFile = { val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) - PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts) + PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts, + file.getModificationTime, file.getLen) } private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 7dfb9168b275..4e1418211188 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{DataType, LongType, StringType, StructField, StructType} /** @@ -173,11 +173,21 @@ trait FileFormat { object FileFormat { + val FILE_PATH = "file_path" + + val FILE_NAME = "file_name" + + val FILE_SIZE = "file_size" + + val FILE_MODIFICATION_TIME = "file_modification_time" + // supported metadata columns for hadoop fs relation val FILE_METADATA_COLUMNS: MetadataAttribute = MetadataAttribute("_metadata", new StructType() - .add(StructField("file_path", StringType)) - .add(StructField("file_name", StringType))) + .add(StructField(FILE_PATH, StringType)) + .add(StructField(FILE_NAME, StringType)) + .add(StructField(FILE_SIZE, LongType)) + .add(StructField(FILE_MODIFICATION_TIME, LongType))) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index fcbf3bdbc318..3a6baa11f6fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -44,14 +44,17 @@ import org.apache.spark.util.NextIterator * @param filePath URI of the file to read * @param start the beginning offset (in bytes) of the block. * @param length number of bytes to read. - * @param locations locality information (list of nodes that have the data). + * @param modificationTime The modification time of the input file, in milliseconds. + * @param fileSize The length of the input file (not the block), in bytes. */ case class PartitionedFile( partitionValues: InternalRow, filePath: String, start: Long, length: Long, - @transient locations: Array[String] = Array.empty) { + @transient locations: Array[String] = Array.empty, + modificationTime: Long = 0L, + fileSize: Long = 0L) { override def toString: String = { s"path: $filePath, range: $start-${start + length}, partition values: $partitionValues" } @@ -144,11 +147,16 @@ class FileScanRDD( metadataStructGenericRow = new GenericRow(1) } else { // make an generic row - metadataStructGenericRow = Row.fromSeq(Seq( - // file_path - UTF8String.fromString(new File(currentFile.filePath).toString), - // file_name - UTF8String.fromString(currentFile.filePath.split("/").last))) + assert(meta.dataType.isInstanceOf[StructType]) + metadataStructGenericRow = Row.fromSeq( + meta.dataType.asInstanceOf[StructType].names.map { + case FILE_PATH => UTF8String.fromString(new File(currentFile.filePath).toString) + case FILE_NAME => UTF8String.fromString( + currentFile.filePath.split("/").last) + case FILE_SIZE => currentFile.fileSize + case FILE_MODIFICATION_TIME => currentFile.modificationTime + } + ) // convert the generic row to an unsafe row val unsafeRowConverter = { @@ -167,19 +175,35 @@ class FileScanRDD( /** * Create a writable column vector containing all required metadata fields */ - private def createMetadataStructColumnVector(c: ColumnarBatch): WritableColumnVector = { + private def createMetadataStructColumnVector( + c: ColumnarBatch, meta: MetadataAttribute): WritableColumnVector = { val columnVector = createColumnVector(c.numRows(), FILE_METADATA_COLUMNS.dataType) val filePathBytes = new File(currentFile.filePath).toString.getBytes val fileNameBytes = currentFile.filePath.split("/").last.getBytes - var rowId = 0 - // use a tight-loop for better performance - while (rowId < c.numRows()) { - // _file_path - columnVector.getChild(0).putByteArray(rowId, filePathBytes) - // _file_name - columnVector.getChild(1).putByteArray(rowId, fileNameBytes) - rowId += 1 + + assert(meta.dataType.isInstanceOf[StructType]) + meta.dataType.asInstanceOf[StructType].names.zipWithIndex.foreach { case (name, ind) => + name match { + case FILE_PATH => + rowId = 0 + // use a tight-loop for better performance + while (rowId < c.numRows()) { + columnVector.getChild(ind).putByteArray(rowId, filePathBytes) + rowId += 1 + } + case FILE_NAME => + rowId = 0 + // use a tight-loop for better performance + while (rowId < c.numRows()) { + columnVector.getChild(ind).putByteArray(rowId, fileNameBytes) + rowId += 1 + } + case FILE_SIZE => + columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.fileSize) + case FILE_MODIFICATION_TIME => + columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.modificationTime) + } } columnVector } @@ -194,7 +218,7 @@ class FileScanRDD( nextElement match { case c: ColumnarBatch => val columnVectorArr = Array.tabulate(c.numCols())(c.column) ++ - Array(createMetadataStructColumnVector(c)) + Array(createMetadataStructColumnVector(c, meta)) new ColumnarBatch(columnVectorArr, c.numRows()) case u: UnsafeRow => val joiner = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala index a8c34ef939de..14408f17d168 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources import java.io.File import java.nio.file.Files +import java.text.SimpleDateFormat import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.functions._ @@ -46,13 +47,17 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { val schemaWithNameConflicts: StructType = new StructType() .add(StructField("name", StringType)) .add(StructField("age", IntegerType)) - .add(StructField("id", LongType)) + .add(StructField("_metadata.file_size", LongType)) .add(StructField("_metadata.FILE_NAME", StringType)) private val METADATA_FILE_PATH = "_metadata.file_path" private val METADATA_FILE_NAME = "_metadata.file_name" + private val METADATA_FILE_SIZE = "_metadata.file_size" + + private val METADATA_FILE_MODIFICATION_TIME = "_metadata.file_modification_time" + /** * Create a CSV file named `fileName` with `data` under `dir` directory. */ @@ -104,17 +109,22 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { val realF0 = new File(dir, "data/f0").listFiles() .filter(_.getName.endsWith(s".$testFileFormat")).head + val realF1 = new File(dir, "data/f1").listFiles() .filter(_.getName.endsWith(s".$testFileFormat")).head // construct f0 and f1 metadata data val f0Metadata = Map( METADATA_FILE_PATH -> realF0.toURI.toString, - METADATA_FILE_NAME -> realF0.getName + METADATA_FILE_NAME -> realF0.getName, + METADATA_FILE_SIZE -> realF0.length(), + METADATA_FILE_MODIFICATION_TIME -> realF0.lastModified() ) val f1Metadata = Map( METADATA_FILE_PATH -> realF1.toURI.toString, - METADATA_FILE_NAME -> realF1.getName + METADATA_FILE_NAME -> realF1.getName, + METADATA_FILE_SIZE -> realF1.length(), + METADATA_FILE_MODIFICATION_TIME -> realF1.lastModified() ) f(df, f0Metadata, f1Metadata) @@ -127,19 +137,22 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { // read all available metadata columns checkAnswer( df.select("name", "age", "id", "university", - METADATA_FILE_NAME, METADATA_FILE_PATH), + METADATA_FILE_NAME, METADATA_FILE_PATH, + METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), Seq( - Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), - Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), + Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) // read a part of metadata columns checkAnswer( - df.select("name", "university", METADATA_FILE_NAME), + df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), Seq( - Row("jack", "uom", f0(METADATA_FILE_NAME)), - Row("lily", "ucb", f1(METADATA_FILE_NAME)) + Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), + Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) ) ) } @@ -147,10 +160,10 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { metadataColumnsTest("read metadata columns with random ordering", schema) { (df, f0, f1) => // read a part of metadata columns with random ordering checkAnswer( - df.select(METADATA_FILE_NAME, "name", METADATA_FILE_PATH, "university"), + df.select(METADATA_FILE_NAME, "name", METADATA_FILE_SIZE, "university"), Seq( - Row(f0(METADATA_FILE_NAME), "jack", f0(METADATA_FILE_PATH), "uom"), - Row(f1(METADATA_FILE_NAME), "lily", f1(METADATA_FILE_PATH), "ucb") + Row(f0(METADATA_FILE_NAME), "jack", f0(METADATA_FILE_SIZE), "uom"), + Row(f1(METADATA_FILE_NAME), "lily", f1(METADATA_FILE_SIZE), "ucb") ) ) } @@ -160,16 +173,25 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { df.select( // substring of file name substring(col(METADATA_FILE_NAME), 1, 3), + // convert timestamp in millis to unixtime and to date format + from_unixtime(col(METADATA_FILE_MODIFICATION_TIME).divide(lit(1000)), "yyyy-MM") + .as("_file_modification_date"), + // convert to kb + col(METADATA_FILE_SIZE).divide(lit(1024)).as("_file_size_kb"), // get the file format substring_index(col(METADATA_FILE_PATH), ".", -1).as("_file_format") ), Seq( Row( f0(METADATA_FILE_NAME).toString.substring(0, 3), // sql substring vs scala substring + new SimpleDateFormat("yyyy-MM").format(f0(METADATA_FILE_MODIFICATION_TIME)), + f0(METADATA_FILE_SIZE).asInstanceOf[Long] / 1024.toDouble, f0(METADATA_FILE_PATH).toString.split("\\.").takeRight(1).head ), Row( f1(METADATA_FILE_NAME).toString.substring(0, 3), // sql substring vs scala substring + new SimpleDateFormat("yyyy-MM").format(f1(METADATA_FILE_MODIFICATION_TIME)), + f1(METADATA_FILE_SIZE).asInstanceOf[Long] / 1024.toDouble, f1(METADATA_FILE_PATH).toString.split("\\.").takeRight(1).head ) ) @@ -186,31 +208,37 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) } - metadataColumnsTest("metadata columns is struct, user data column is string", - schemaWithNameConflicts) { (df, f0, f1) => - // here: the data has the schema: name, age, id, _metadata.file_name + metadataColumnsTest("metadata columns will not " + + "overwrite user data schema", schemaWithNameConflicts) { (df, f0, f1) => + // here: the data has the schema: name, age, _metadata.file_size, _metadata.file_name checkAnswer( - df.select("name", "age", "id", "`_metadata.FILE_NAME`", - METADATA_FILE_NAME, METADATA_FILE_PATH), + df.select("name", "age", "`_metadata.file_size`", "`_metadata.FILE_NAME`", + METADATA_FILE_NAME, METADATA_FILE_PATH, + METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), Seq( Row("jack", 24, 12345L, "uom", - // user data will not be overwritten, + // uom and 12345L will not be overwritten, // and we still can read metadata columns correctly - f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), + f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), Row("lily", 31, null, "ucb", - // user data will not be overwritten, + // ucb and `null` will not be overwritten, // and we still can read metadata columns correctly - f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) } metadataColumnsTest("select only metadata columns", schema) { (df, f0, f1) => checkAnswer( - df.select(METADATA_FILE_NAME, METADATA_FILE_PATH), + df.select(METADATA_FILE_NAME, METADATA_FILE_PATH, + METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), Seq( - Row(f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), - Row(f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + Row(f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), + Row(f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) } @@ -220,7 +248,8 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { // to the more general internal row, thus it will fail to re-select checkAnswer( df.select("name", "age", "id", "university", - METADATA_FILE_NAME, METADATA_FILE_PATH) + METADATA_FILE_NAME, METADATA_FILE_PATH, + METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME) .select("name", "file_path"), // cast _metadata.file_path as file_path Seq( Row("jack", f0(METADATA_FILE_PATH)), @@ -234,7 +263,8 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { val aliasDF = df.select( Column("name").as("myName"), Column("age").as("myAge"), - Column(METADATA_FILE_NAME).as("myFileName") + Column(METADATA_FILE_NAME).as("myFileName"), + Column(METADATA_FILE_SIZE).as("myFileSize") ) // check schema @@ -242,6 +272,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { .add(StructField("myName", StringType)) .add(StructField("myAge", IntegerType)) .add(StructField("myFileName", StringType)) + .add(StructField("myFileSize", LongType)) assert(aliasDF.schema.fields.toSet == expectedSchema.fields.toSet) @@ -249,8 +280,8 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { checkAnswer( aliasDF, Seq( - Row("jack", 24, f0(METADATA_FILE_NAME)), - Row("lily", 31, f1(METADATA_FILE_NAME)) + Row("jack", 24, f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), + Row("lily", 31, f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) ) ) } @@ -271,40 +302,49 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { s"sensitive is $caseSensitive", schemaWithNameConflicts) { (df, f0, f1) => withSQLConf("spark.sql.caseSensitive" -> caseSensitive.toString) { - // file schema: name, age, id, _metadata._FILE_NAME + // file schema: name, age, _file_size, _FILE_NAME if (caseSensitive) { // for case sensitive mode: - // _METADATA.FILE_PATH is not a part of user schema or metadata columns + // _METADATA.FILE_SIZE is not a part of user schema or metadata columns val ex = intercept[Exception] { - df.select("name", "age", "_METADATA.FILE_PATH").show() + df.select("name", "age", "_METADATA.FILE_SIZE").show() } - assert(ex.getMessage.contains("_METADATA.FILE_PATH")) + assert(ex.getMessage.contains("_METADATA.FILE_SIZE")) // for case sensitive mode: - // `_metadata.FILE_NAME` is in the user schema - // _metadata.file_name is metadata columns + // `_metadata.file_size` and `_metadata.FILE_NAME` are in the user schema + // _metadata.file_name and _metadata.file_modification_time are metadata columns checkAnswer( - df.select("name", "age", "id", "`_metadata.FILE_NAME`", - "_metadata.file_name"), + df.select("name", "age", "`_metadata.file_size`", "`_metadata.FILE_NAME`", + "_metadata.file_name", "_metadata.file_modification_time"), Seq( - Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME)), - Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME)) + Row("jack", 24, 12345L, "uom", + f0(METADATA_FILE_NAME), f0(METADATA_FILE_MODIFICATION_TIME)), + Row("lily", 31, null, "ucb", + f1(METADATA_FILE_NAME), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) } else { // for case insensitive mode: + // `_metadata.file_size`, `_metadata.FILE_SIZE`, // `_metadata.file_name`, `_metadata.FILE_NAME` are all from the user schema. - // different casings of _metadata.file_path is metadata columns + // different casings of _metadata.file_path and + // _metadata.file_modification_time are metadata columns checkAnswer( - df.select("name", "age", "id", + df.select("name", "age", + // user columns + "`_metadata.file_size`", "`_metadata.FILE_SIZE`", "`_metadata.file_name`", "`_metadata.FILE_NAME`", // metadata columns - "_metadata.file_path", "_metadata.FILE_PATH"), + "_metadata.file_path", "_metadata.FILE_PATH", + "_metadata.file_modification_time", "_metadata.FILE_modification_TiMe"), Seq( - Row("jack", 24, 12345L, "uom", "uom", - f0(METADATA_FILE_PATH), f0(METADATA_FILE_PATH)), - Row("lily", 31, null, "ucb", "ucb", - f1(METADATA_FILE_PATH), f1(METADATA_FILE_PATH)) + Row("jack", 24, 12345L, 12345L, "uom", "uom", + f0(METADATA_FILE_PATH), f0(METADATA_FILE_PATH), + f0(METADATA_FILE_MODIFICATION_TIME), f0(METADATA_FILE_MODIFICATION_TIME)), + Row("lily", 31, null, null, "ucb", "ucb", + f1(METADATA_FILE_PATH), f1(METADATA_FILE_PATH), + f1(METADATA_FILE_MODIFICATION_TIME), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) } @@ -312,28 +352,35 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { } } - Seq("true", "false").foreach { offHeapColumnVectorEnabled => - withSQLConf("spark.sql.columnVector.offheap.enabled" -> offHeapColumnVectorEnabled) { - metadataColumnsTest(s"read metadata with " + - s"offheap set to $offHeapColumnVectorEnabled", schema) { (df, f0, f1) => - // read all available metadata columns - checkAnswer( - df.select("name", "age", "id", "university", - METADATA_FILE_NAME, METADATA_FILE_PATH), - Seq( - Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH)), - Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH)) + Seq("true", "false").foreach { photonEnabled => + Seq("true", "false").foreach { offHeapColumnVectorEnabled => + withSQLConf("spark.sql.columnVector.offheap.enabled" -> offHeapColumnVectorEnabled, + "spark.databricks.photon.enabled" -> photonEnabled) { + metadataColumnsTest(s"read metadata with " + + s"offheap set to $offHeapColumnVectorEnabled, " + + s"photon set to $photonEnabled", schema) { (df, f0, f1) => + // read all available metadata columns + checkAnswer( + df.select("name", "age", "id", "university", + METADATA_FILE_NAME, METADATA_FILE_PATH, + METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), + Seq( + Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), + Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) + ) ) - ) - // read a part of metadata columns - checkAnswer( - df.select("name", "university", METADATA_FILE_NAME), - Seq( - Row("jack", "uom", f0(METADATA_FILE_NAME)), - Row("lily", "ucb", f1(METADATA_FILE_NAME)) + // read a part of metadata columns + checkAnswer( + df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), + Seq( + Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), + Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) + ) ) - ) + } } } } @@ -410,18 +457,23 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { val f1 = new File(dir, "temp/1") val metadata = Map( METADATA_FILE_PATH -> f1.toURI.toString, - METADATA_FILE_NAME -> f1.getName + METADATA_FILE_NAME -> f1.getName, + METADATA_FILE_SIZE -> f1.length(), + METADATA_FILE_MODIFICATION_TIME -> f1.lastModified() ) checkAnswer( df1.select("name", "metadata.file_name", "_metadata.file_path", "_metadata.file_name", + "_metadata.file_size", "_metadata.file_modification_time", "_metadata"), Row("jack", "jack.json", metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME), + metadata(METADATA_FILE_SIZE), metadata(METADATA_FILE_MODIFICATION_TIME), // struct of _metadata Row( - metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME)) + metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME), + metadata(METADATA_FILE_SIZE), metadata(METADATA_FILE_MODIFICATION_TIME)) ) ) } From 170378bc4747be2d311d420b74e8065b5eef9086 Mon Sep 17 00:00:00 2001 From: yaohua Date: Mon, 15 Nov 2021 15:40:08 -0800 Subject: [PATCH 04/21] metadata unapply --- .../expressions/namedExpressions.scala | 73 +++---------------- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 20 +++-- .../execution/datasources/FileFormat.scala | 2 +- .../execution/datasources/FileScanRDD.scala | 6 +- .../datasources/FileSourceStrategy.scala | 2 +- 6 files changed, 32 insertions(+), 73 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 40f40f028665..9e984d3558df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.trees.TreePattern import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.catalyst.util.quoteIfNeeded +import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, METADATA_COL_ATTR_KEY} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.types._ import org.apache.spark.util.collection.BitSet @@ -442,66 +442,15 @@ object VirtualColumn { /** * The internal representation of the hidden metadata column */ -class MetadataAttribute( - override val name: String, - override val dataType: DataType, - override val nullable: Boolean = true, - override val metadata: Metadata = Metadata.empty)( - override val exprId: ExprId = NamedExpression.newExprId, - override val qualifier: Seq[String] = Seq.empty[String]) - extends AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier) { - - // use to resolve supported metadata column references (e.g. different casings) - override def withName(newName: String): MetadataAttribute = { - if (name == newName) { - this - } else { - MetadataAttribute(newName, dataType, nullable, metadata)(exprId, qualifier) - } - } - - override def withNullability(newNullability: Boolean): MetadataAttribute = { - if (nullable == newNullability) { - this - } else { - MetadataAttribute(name, dataType, newNullability, metadata)(exprId, qualifier) - } - } - - override def withQualifier(newQualifier: Seq[String]): MetadataAttribute = { - if (qualifier == newQualifier) { - this - } else { - MetadataAttribute(name, dataType, nullable, metadata)(exprId, newQualifier) - } - } - - override def withExprId(newExprId: ExprId): MetadataAttribute = { - if (exprId == newExprId) { - this - } else { - MetadataAttribute(name, dataType, nullable, metadata)(newExprId, qualifier) - } - } - - override def withDataType(newType: DataType): MetadataAttribute = { - MetadataAttribute(name, newType, nullable, metadata)(exprId, qualifier) - } - - override def newInstance(): MetadataAttribute = - MetadataAttribute(name, dataType, nullable, metadata)(exprId, qualifier) - - override def withMetadata(newMetadata: Metadata): MetadataAttribute = { - MetadataAttribute(name, dataType, nullable, newMetadata)(exprId, qualifier) - } -} - object MetadataAttribute { - - def apply(name: String, dataType: DataType): MetadataAttribute = - new MetadataAttribute(name, dataType, true)() - - def apply(name: String, dataType: DataType, nullable: Boolean, metadata: Metadata) - (exprId: ExprId, qualifier: Seq[String]): MetadataAttribute = - new MetadataAttribute(name, dataType, nullable, metadata)(exprId, qualifier) + def apply(name: String, dataType: DataType): AttributeReference = + AttributeReference(name, dataType, true, + new MetadataBuilder().putBoolean(METADATA_COL_ATTR_KEY, true).build())() + + def unapply(attr: AttributeReference): Option[AttributeReference] = { + if (attr.metadata.contains(METADATA_COL_ATTR_KEY) + && attr.metadata.getBoolean(METADATA_COL_ATTR_KEY)) { + Some(attr) + } else None + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 66ef9650b5ee..03aec625e973 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -278,7 +278,7 @@ object LogicalPlanIntegrity { } /** - * A logical plan node with exposed metadata columns + * A logical plan node that can generate metadata columns */ trait ExposesMetadataColumns extends LogicalPlan { def withMetadataColumns(): ExposesMetadataColumns diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 81dbb89c2b1e..0904bdb17eaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -195,15 +195,20 @@ case class FileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { - lazy val outputMetadataStruct: Option[MetadataAttribute] = - output.collectFirst { case meta: MetadataAttribute => meta } + lazy val outputMetadataStruct: Option[AttributeReference] = + output.collectFirst { case MetadataAttribute(attr) => attr } // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. override lazy val supportsColumnar: Boolean = { - // schema without file metadata column + // schema without the file metadata column val fileSchema = if (outputMetadataStruct.isEmpty) schema else { - StructType.fromAttributes(output.filterNot(_.isInstanceOf[MetadataAttribute])) + StructType.fromAttributes( + output.filter { + case MetadataAttribute(_) => false + case _ => true + } + ) } relation.fileFormat.supportBatch(relation.sparkSession, fileSchema) } @@ -373,7 +378,12 @@ case class FileSourceScanExec( private lazy val pushedDownFilters = { val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) dataFilters - .filterNot(_.references.exists(_.isInstanceOf[MetadataAttribute])) + .filterNot( + _.references.exists { + case MetadataAttribute(_) => true + case _ => false + } + ) .flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 4e1418211188..f37233ce1a66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -182,7 +182,7 @@ object FileFormat { val FILE_MODIFICATION_TIME = "file_modification_time" // supported metadata columns for hadoop fs relation - val FILE_METADATA_COLUMNS: MetadataAttribute = MetadataAttribute("_metadata", + val FILE_METADATA_COLUMNS: AttributeReference = MetadataAttribute("_metadata", new StructType() .add(StructField(FILE_PATH, StringType)) .add(StructField(FILE_NAME, StringType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 3a6baa11f6fc..9e47087f5e1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -26,7 +26,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{GenericRow, MetadataAttribute, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.FileFormat._ @@ -68,7 +68,7 @@ class FileScanRDD( readFunction: (PartitionedFile) => Iterator[InternalRow], @transient val filePartitions: Seq[FilePartition], val requiredSchema: StructType = StructType(Seq.empty), - val metadataStruct: Option[MetadataAttribute] = None) + val metadataStruct: Option[AttributeReference] = None) extends RDD[InternalRow](sparkSession.sparkContext, Nil) { private val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles @@ -176,7 +176,7 @@ class FileScanRDD( * Create a writable column vector containing all required metadata fields */ private def createMetadataStructColumnVector( - c: ColumnarBatch, meta: MetadataAttribute): WritableColumnVector = { + c: ColumnarBatch, meta: AttributeReference): WritableColumnVector = { val columnVector = createColumnVector(c.numRows(), FILE_METADATA_COLUMNS.dataType) val filePathBytes = new File(currentFile.filePath).toString.getBytes val fileNameBytes = currentFile.filePath.split("/").last.getBytes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 75efb432e94b..f43551a0a29e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -214,7 +214,7 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { // outputAttributes should also include referenced metadata columns at the every end val outputAttributes = readDataColumns ++ partitionColumns ++ - plan.references.collect { case meta: MetadataAttribute => meta } + plan.references.collect { case MetadataAttribute(attr) => attr } val scan = FileSourceScanExec( From 73593c5a9a26889709b57e938a3f6efca14ab1fb Mon Sep 17 00:00:00 2001 From: yaohua Date: Mon, 15 Nov 2021 15:47:14 -0800 Subject: [PATCH 05/21] nits --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 5 ++++- .../apache/spark/sql/execution/datasources/FileFormat.scala | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 9e984d3558df..87fb9db43cba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -440,7 +440,10 @@ object VirtualColumn { } /** - * The internal representation of the hidden metadata column + * The internal representation of the hidden metadata column: + * set `__metadata_col` to `true` in AttributeReference metadata + * - apply() will create a metadata attribute reference + * - unapply() will check if an attribute reference is the metadata attribute reference */ object MetadataAttribute { def apply(name: String, dataType: DataType): AttributeReference = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index f37233ce1a66..850c6fdb9062 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -181,8 +181,10 @@ object FileFormat { val FILE_MODIFICATION_TIME = "file_modification_time" + val METADATA_NAME = "_metadata" + // supported metadata columns for hadoop fs relation - val FILE_METADATA_COLUMNS: AttributeReference = MetadataAttribute("_metadata", + val FILE_METADATA_COLUMNS: AttributeReference = MetadataAttribute(METADATA_NAME, new StructType() .add(StructField(FILE_PATH, StringType)) .add(StructField(FILE_NAME, StringType)) From c5313007953510175a3275421ce8c040ee9669e3 Mon Sep 17 00:00:00 2001 From: yaohua Date: Mon, 15 Nov 2021 17:47:41 -0800 Subject: [PATCH 06/21] exhausive pattern match --- .../apache/spark/sql/execution/datasources/FileScanRDD.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 9e47087f5e1d..139af9da81c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -155,6 +155,7 @@ class FileScanRDD( currentFile.filePath.split("/").last) case FILE_SIZE => currentFile.fileSize case FILE_MODIFICATION_TIME => currentFile.modificationTime + case _ => None // just be exhaustive, won't happen } ) @@ -203,6 +204,7 @@ class FileScanRDD( columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.fileSize) case FILE_MODIFICATION_TIME => columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.modificationTime) + case _ => // just be exhaustive, won't happen: no-op } } columnVector From bd28eb7075c3fbd8f003ad7ade1f596795e4ef2a Mon Sep 17 00:00:00 2001 From: yaohua Date: Mon, 15 Nov 2021 21:10:08 -0800 Subject: [PATCH 07/21] nit refactor --- .../execution/datasources/FileScanRDD.scala | 92 ++++++++++--------- 1 file changed, 47 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 139af9da81c9..b21ddd1b901c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -140,37 +140,37 @@ class FileScanRDD( * Only update metadata columns when `currentFile` is changed. */ private def updateMetadataStruct(): Unit = - metadataStruct.foreach { - case meta if FILE_METADATA_COLUMNS.sameRef(meta) => - if (currentFile == null) { - metadataStructUnsafeRow = new UnsafeRow(1) - metadataStructGenericRow = new GenericRow(1) - } else { - // make an generic row - assert(meta.dataType.isInstanceOf[StructType]) - metadataStructGenericRow = Row.fromSeq( - meta.dataType.asInstanceOf[StructType].names.map { - case FILE_PATH => UTF8String.fromString(new File(currentFile.filePath).toString) - case FILE_NAME => UTF8String.fromString( - currentFile.filePath.split("/").last) - case FILE_SIZE => currentFile.fileSize - case FILE_MODIFICATION_TIME => currentFile.modificationTime - case _ => None // just be exhaustive, won't happen - } - ) - - // convert the generic row to an unsafe row - val unsafeRowConverter = { - val converter = UnsafeProjection.create( - Array(FILE_METADATA_COLUMNS.dataType)) - (row: Row) => { - converter(CatalystTypeConverters.convertToCatalyst(row) - .asInstanceOf[InternalRow]) - } + if (metadataStruct.exists(_.sameRef(FILE_METADATA_COLUMNS))) { + val meta = metadataStruct.get + if (currentFile == null) { + metadataStructUnsafeRow = new UnsafeRow(1) + metadataStructGenericRow = new GenericRow(1) + } else { + // make an generic row + assert(meta.dataType.isInstanceOf[StructType]) + metadataStructGenericRow = Row.fromSeq( + meta.dataType.asInstanceOf[StructType].names.map { + case FILE_PATH => UTF8String.fromString(new File(currentFile.filePath).toString) + case FILE_NAME => UTF8String.fromString( + currentFile.filePath.split("/").last) + case FILE_SIZE => currentFile.fileSize + case FILE_MODIFICATION_TIME => currentFile.modificationTime + case _ => None // be exhaustive, won't happen + } + ) + + // convert the generic row to an unsafe row + val unsafeRowConverter = { + val converter = UnsafeProjection.create( + Array(FILE_METADATA_COLUMNS.dataType)) + (row: Row) => { + converter(CatalystTypeConverters.convertToCatalyst(row) + .asInstanceOf[InternalRow]) } - metadataStructUnsafeRow = - unsafeRowConverter(Row.fromSeq(Seq(metadataStructGenericRow))) } + metadataStructUnsafeRow = + unsafeRowConverter(Row.fromSeq(Seq(metadataStructGenericRow))) + } } /** @@ -204,7 +204,7 @@ class FileScanRDD( columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.fileSize) case FILE_MODIFICATION_TIME => columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.modificationTime) - case _ => // just be exhaustive, won't happen: no-op + case _ => // be exhaustive, won't happen } } columnVector @@ -215,21 +215,23 @@ class FileScanRDD( * For different row implementations, use different methods to update and append. */ private def addMetadataStructIfNeeded(nextElement: Object): Object = - metadataStruct.map { - case meta if FILE_METADATA_COLUMNS.sameRef(meta) => - nextElement match { - case c: ColumnarBatch => - val columnVectorArr = Array.tabulate(c.numCols())(c.column) ++ - Array(createMetadataStructColumnVector(c, meta)) - new ColumnarBatch(columnVectorArr, c.numRows()) - case u: UnsafeRow => - val joiner = - GenerateUnsafeRowJoiner.create(requiredSchema, Seq(meta).toStructType) - joiner.join(u, metadataStructUnsafeRow) - case i: InternalRow => - InternalRow.fromSeq(i.toSeq(requiredSchema) ++ metadataStructGenericRow.toSeq) - } - }.getOrElse(nextElement) + if (metadataStruct.exists(_.sameRef(FILE_METADATA_COLUMNS))) { + val meta = metadataStruct.get + nextElement match { + case c: ColumnarBatch => + val columnVectorArr = Array.tabulate(c.numCols())(c.column) ++ + Array(createMetadataStructColumnVector(c, meta)) + new ColumnarBatch(columnVectorArr, c.numRows()) + case u: UnsafeRow => + val joiner = + GenerateUnsafeRowJoiner.create(requiredSchema, Seq(meta).toStructType) + joiner.join(u, metadataStructUnsafeRow) + case i: InternalRow => + InternalRow.fromSeq(i.toSeq(requiredSchema) ++ metadataStructGenericRow.toSeq) + } + } else { + nextElement + } def next(): Object = { val nextElement = currentIterator.next() From e872d1fae7acde9445dfac389e430d92406f9e09 Mon Sep 17 00:00:00 2001 From: yaohua Date: Sun, 21 Nov 2021 16:28:50 -0800 Subject: [PATCH 08/21] addressed comments --- .../expressions/namedExpressions.scala | 6 +- .../execution/datasources/FileFormat.scala | 13 +++-- .../execution/datasources/FileScanRDD.scala | 44 +++++++++------ .../FileMetadataColumnsSuite.scala | 55 +++++++++---------- 4 files changed, 65 insertions(+), 53 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 87fb9db43cba..70fe5bf81c35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -446,9 +446,9 @@ object VirtualColumn { * - unapply() will check if an attribute reference is the metadata attribute reference */ object MetadataAttribute { - def apply(name: String, dataType: DataType): AttributeReference = - AttributeReference(name, dataType, true, - new MetadataBuilder().putBoolean(METADATA_COL_ATTR_KEY, true).build())() + def apply(name: String, dataType: DataType, nullable: Boolean = true): AttributeReference = + AttributeReference(name, dataType, nullable, + new MetadataBuilder().putBoolean(METADATA_COL_ATTR_KEY, value = true).build())() def unapply(attr: AttributeReference): Option[AttributeReference] = { if (attr.metadata.contains(METADATA_COL_ATTR_KEY) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 850c6fdb9062..54606afa8e03 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -183,13 +183,14 @@ object FileFormat { val METADATA_NAME = "_metadata" + val METADATA_STRUCT: DataType = new StructType() + .add(StructField(FILE_PATH, StringType)) + .add(StructField(FILE_NAME, StringType)) + .add(StructField(FILE_SIZE, LongType)) + .add(StructField(FILE_MODIFICATION_TIME, LongType)) + // supported metadata columns for hadoop fs relation - val FILE_METADATA_COLUMNS: AttributeReference = MetadataAttribute(METADATA_NAME, - new StructType() - .add(StructField(FILE_PATH, StringType)) - .add(StructField(FILE_NAME, StringType)) - .add(StructField(FILE_SIZE, LongType)) - .add(StructField(FILE_MODIFICATION_TIME, LongType))) + def FILE_METADATA_COLUMNS: AttributeReference = MetadataAttribute(METADATA_NAME, METADATA_STRUCT) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index b21ddd1b901c..9657dd9dc266 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -26,8 +26,8 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, JoinedRow, MetadataAttribute, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeRowJoiner, UnsafeRowJoiner} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.FileFormat._ import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} @@ -121,10 +121,22 @@ class FileScanRDD( // FILE METADATA METHODS // /////////////////////////// - // use to join with an unsafe row, will only be updated when the current file is changed - @volatile var metadataStructUnsafeRow: UnsafeRow = _ - // use to append to an internal row, will only be updated when the current file is changed - @volatile var metadataStructGenericRow: Row = _ + // whether a metadata column exists and it is a `MetadataAttribute` + private lazy val hasMetadataAttribute: Boolean = { + metadataStruct.exists { + case MetadataAttribute(_) => true + case _ => false + } + } + + // metadata struct unsafe row, will only be updated when the current file is changed + @volatile private var metadataStructUnsafeRow: UnsafeRow = _ + // metadata generic row, will only be updated when the current file is changed + @volatile private var metadataStructGenericRow: Row = _ + // an unsafe joiner to join an unsafe row with the metadata unsafe row + lazy private val unsafeRowJoiner = + if (hasMetadataAttribute) + GenerateUnsafeRowJoiner.create(requiredSchema, Seq(metadataStruct.get).toStructType) // Create a off/on heap WritableColumnVector private def createColumnVector(numRows: Int, dataType: DataType): WritableColumnVector = { @@ -139,8 +151,8 @@ class FileScanRDD( * For each partitioned file, metadata columns for each record in the file are exactly same. * Only update metadata columns when `currentFile` is changed. */ - private def updateMetadataStruct(): Unit = - if (metadataStruct.exists(_.sameRef(FILE_METADATA_COLUMNS))) { + private def updateMetadataStruct(): Unit = { + if (hasMetadataAttribute) { val meta = metadataStruct.get if (currentFile == null) { metadataStructUnsafeRow = new UnsafeRow(1) @@ -162,7 +174,7 @@ class FileScanRDD( // convert the generic row to an unsafe row val unsafeRowConverter = { val converter = UnsafeProjection.create( - Array(FILE_METADATA_COLUMNS.dataType)) + Array(METADATA_STRUCT)) (row: Row) => { converter(CatalystTypeConverters.convertToCatalyst(row) .asInstanceOf[InternalRow]) @@ -172,13 +184,14 @@ class FileScanRDD( unsafeRowConverter(Row.fromSeq(Seq(metadataStructGenericRow))) } } + } /** * Create a writable column vector containing all required metadata fields */ private def createMetadataStructColumnVector( c: ColumnarBatch, meta: AttributeReference): WritableColumnVector = { - val columnVector = createColumnVector(c.numRows(), FILE_METADATA_COLUMNS.dataType) + val columnVector = createColumnVector(c.numRows(), METADATA_STRUCT) val filePathBytes = new File(currentFile.filePath).toString.getBytes val fileNameBytes = currentFile.filePath.split("/").last.getBytes var rowId = 0 @@ -214,8 +227,8 @@ class FileScanRDD( * Add metadata struct at the end of nextElement if needed. * For different row implementations, use different methods to update and append. */ - private def addMetadataStructIfNeeded(nextElement: Object): Object = - if (metadataStruct.exists(_.sameRef(FILE_METADATA_COLUMNS))) { + private def addMetadataStructIfNeeded(nextElement: Object): Object = { + if (hasMetadataAttribute) { val meta = metadataStruct.get nextElement match { case c: ColumnarBatch => @@ -223,15 +236,14 @@ class FileScanRDD( Array(createMetadataStructColumnVector(c, meta)) new ColumnarBatch(columnVectorArr, c.numRows()) case u: UnsafeRow => - val joiner = - GenerateUnsafeRowJoiner.create(requiredSchema, Seq(meta).toStructType) - joiner.join(u, metadataStructUnsafeRow) + unsafeRowJoiner.asInstanceOf[UnsafeRowJoiner].join(u, metadataStructUnsafeRow) case i: InternalRow => - InternalRow.fromSeq(i.toSeq(requiredSchema) ++ metadataStructGenericRow.toSeq) + new JoinedRow(i, InternalRow.fromSeq(metadataStructGenericRow.toSeq)) } } else { nextElement } + } def next(): Object = { val nextElement = currentIterator.next() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala index 14408f17d168..2d6fc8d74fbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala @@ -44,6 +44,9 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { .add(StructField("id", LongType)) .add(StructField("university", StringType)) + // make sure: + // `_metadata.file_size` (a flat column) and _metadata.file_size (a field in a strut) + // are not the same columns, users could read both correctly val schemaWithNameConflicts: StructType = new StructType() .add(StructField("name", StringType)) .add(StructField("age", IntegerType)) @@ -208,8 +211,8 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) } - metadataColumnsTest("metadata columns will not " + - "overwrite user data schema", schemaWithNameConflicts) { (df, f0, f1) => + metadataColumnsTest("metadata fields and `_metadata.file_size` " + + "are not the same", schemaWithNameConflicts) { (df, f0, f1) => // here: the data has the schema: name, age, _metadata.file_size, _metadata.file_name checkAnswer( df.select("name", "age", "`_metadata.file_size`", "`_metadata.FILE_NAME`", @@ -352,35 +355,31 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { } } - Seq("true", "false").foreach { photonEnabled => - Seq("true", "false").foreach { offHeapColumnVectorEnabled => - withSQLConf("spark.sql.columnVector.offheap.enabled" -> offHeapColumnVectorEnabled, - "spark.databricks.photon.enabled" -> photonEnabled) { - metadataColumnsTest(s"read metadata with " + - s"offheap set to $offHeapColumnVectorEnabled, " + - s"photon set to $photonEnabled", schema) { (df, f0, f1) => - // read all available metadata columns - checkAnswer( - df.select("name", "age", "id", "university", - METADATA_FILE_NAME, METADATA_FILE_PATH, - METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), - Seq( - Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), - f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), - Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), - f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) - ) + Seq("true", "false").foreach { offHeapColumnVectorEnabled => + withSQLConf("spark.sql.columnVector.offheap.enabled" -> offHeapColumnVectorEnabled) { + metadataColumnsTest(s"read metadata with " + + s"offheap set to $offHeapColumnVectorEnabled", schema) { (df, f0, f1) => + // read all available metadata columns + checkAnswer( + df.select("name", "age", "id", "university", + METADATA_FILE_NAME, METADATA_FILE_PATH, + METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), + Seq( + Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), + Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) + ) - // read a part of metadata columns - checkAnswer( - df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), - Seq( - Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), - Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) - ) + // read a part of metadata columns + checkAnswer( + df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), + Seq( + Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), + Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) ) - } + ) } } } From 60bdbc56540e04f0fbaf665cfd85faec5b40f41b Mon Sep 17 00:00:00 2001 From: yaohua Date: Wed, 24 Nov 2021 17:00:46 -0800 Subject: [PATCH 09/21] some nits comments --- .../expressions/namedExpressions.scala | 2 +- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../sql/execution/DataSourceScanExec.scala | 31 +++------ .../execution/datasources/FileFormat.scala | 7 +- .../execution/datasources/FileScanRDD.scala | 67 +++++++------------ .../datasources/FileSourceStrategy.scala | 4 +- .../datasources/LogicalRelation.scala | 7 +- ...te.scala => FileMetadataStructSuite.scala} | 36 +++++----- 8 files changed, 65 insertions(+), 91 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/{FileMetadataColumnsSuite.scala => FileMetadataStructSuite.scala} (93%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 70fe5bf81c35..c18ab7b1bc0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -440,7 +440,7 @@ object VirtualColumn { } /** - * The internal representation of the hidden metadata column: + * The internal representation of the hidden metadata struct: * set `__metadata_col` to `true` in AttributeReference metadata * - apply() will create a metadata attribute reference * - unapply() will check if an attribute reference is the metadata attribute reference diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 03aec625e973..9cff07bab7a4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -281,5 +281,5 @@ object LogicalPlanIntegrity { * A logical plan node that can generate metadata columns */ trait ExposesMetadataColumns extends LogicalPlan { - def withMetadataColumns(): ExposesMetadataColumns + def withMetadataColumns(): LogicalPlan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 0904bdb17eaf..cbb37857514f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.datasources.v2.PushedDownOperators import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector} +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.StructType @@ -195,20 +195,15 @@ case class FileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { - lazy val outputMetadataStruct: Option[AttributeReference] = + lazy val metadataStructCol: Option[AttributeReference] = output.collectFirst { case MetadataAttribute(attr) => attr } // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. override lazy val supportsColumnar: Boolean = { - // schema without the file metadata column - val fileSchema = if (outputMetadataStruct.isEmpty) schema else { - StructType.fromAttributes( - output.filter { - case MetadataAttribute(_) => false - case _ => true - } - ) + // schema without the file metadata struct column + val fileSchema = if (metadataStructCol.isEmpty) schema else { + output.filter(_.exprId != metadataStructCol.get.exprId).toStructType } relation.fileFormat.supportBatch(relation.sparkSession, fileSchema) } @@ -226,14 +221,10 @@ case class FileSourceScanExec( requiredSchema = requiredSchema, partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf).map { vectorTypes => - val metadataVectorClz = - if (relation.sparkSession.sessionState.conf.offHeapColumnVectorEnabled) { - classOf[OffHeapColumnVector].getName - } else { - classOf[OnHeapColumnVector].getName - } - // for column-based file format, append metadata columns' vector type classes if any - vectorTypes ++ (if (outputMetadataStruct.isDefined) Seq(metadataVectorClz) else Seq.empty) + // for column-based file format, append metadata struct column's vector type classes if any + vectorTypes ++ (if (metadataStructCol.isDefined) { + Seq(classOf[OnHeapColumnVector].getName) + } else Seq.empty) } private lazy val driverMetrics: HashMap[String, Long] = HashMap.empty @@ -627,7 +618,7 @@ case class FileSourceScanExec( } new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions, - requiredSchema, outputMetadataStruct) + requiredSchema, metadataStructCol) } /** @@ -684,7 +675,7 @@ case class FileSourceScanExec( FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions, - requiredSchema, outputMetadataStruct) + requiredSchema, metadataStructCol) } // Filters unused DynamicPruningExpression expressions - one which has been replaced diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 54606afa8e03..538e4e3f14f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -183,14 +183,15 @@ object FileFormat { val METADATA_NAME = "_metadata" - val METADATA_STRUCT: DataType = new StructType() + // supported metadata struct fields for hadoop fs relation + val METADATA_STRUCT: StructType = new StructType() .add(StructField(FILE_PATH, StringType)) .add(StructField(FILE_NAME, StringType)) .add(StructField(FILE_SIZE, LongType)) .add(StructField(FILE_MODIFICATION_TIME, LongType)) - // supported metadata columns for hadoop fs relation - def FILE_METADATA_COLUMNS: AttributeReference = MetadataAttribute(METADATA_NAME, METADATA_STRUCT) + // create a file metadata struct col + def createFileMetadataCol: AttributeReference = MetadataAttribute(METADATA_NAME, METADATA_STRUCT) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 9657dd9dc266..4b9b1225fb62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -26,12 +26,12 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, JoinedRow, MetadataAttribute, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeRowJoiner, UnsafeRowJoiner} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.FileFormat._ -import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.NextIterator @@ -68,13 +68,11 @@ class FileScanRDD( readFunction: (PartitionedFile) => Iterator[InternalRow], @transient val filePartitions: Seq[FilePartition], val requiredSchema: StructType = StructType(Seq.empty), - val metadataStruct: Option[AttributeReference] = None) + val metadataStructCol: Option[AttributeReference] = None) extends RDD[InternalRow](sparkSession.sparkContext, Nil) { private val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles private val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles - private val offHeapColumnVectorEnabled = - sparkSession.sessionState.conf.offHeapColumnVectorEnabled override def compute(split: RDDPartition, context: TaskContext): Iterator[InternalRow] = { val iterator = new Iterator[Object] with AutoCloseable { @@ -121,50 +119,34 @@ class FileScanRDD( // FILE METADATA METHODS // /////////////////////////// - // whether a metadata column exists and it is a `MetadataAttribute` - private lazy val hasMetadataAttribute: Boolean = { - metadataStruct.exists { - case MetadataAttribute(_) => true - case _ => false - } - } - // metadata struct unsafe row, will only be updated when the current file is changed - @volatile private var metadataStructUnsafeRow: UnsafeRow = _ + @volatile private var metadataStructColUnsafeRow: UnsafeRow = _ // metadata generic row, will only be updated when the current file is changed - @volatile private var metadataStructGenericRow: Row = _ + @volatile private var metadataStructColGenericRow: Row = _ // an unsafe joiner to join an unsafe row with the metadata unsafe row lazy private val unsafeRowJoiner = - if (hasMetadataAttribute) - GenerateUnsafeRowJoiner.create(requiredSchema, Seq(metadataStruct.get).toStructType) - - // Create a off/on heap WritableColumnVector - private def createColumnVector(numRows: Int, dataType: DataType): WritableColumnVector = { - if (offHeapColumnVectorEnabled) { - new OffHeapColumnVector(numRows, dataType) - } else { - new OnHeapColumnVector(numRows, dataType) + if (metadataStructCol.isDefined) { + GenerateUnsafeRowJoiner.create(requiredSchema, Seq(metadataStructCol.get).toStructType) } - } /** - * For each partitioned file, metadata columns for each record in the file are exactly same. - * Only update metadata columns when `currentFile` is changed. + * For each partitioned file, metadata struct for each record in the file are exactly same. + * Only update metadata struct when `currentFile` is changed. */ private def updateMetadataStruct(): Unit = { - if (hasMetadataAttribute) { - val meta = metadataStruct.get + if (metadataStructCol.isDefined) { + val meta = metadataStructCol.get if (currentFile == null) { - metadataStructUnsafeRow = new UnsafeRow(1) - metadataStructGenericRow = new GenericRow(1) + metadataStructColUnsafeRow = null + metadataStructColGenericRow = null } else { // make an generic row assert(meta.dataType.isInstanceOf[StructType]) - metadataStructGenericRow = Row.fromSeq( + metadataStructColGenericRow = Row.fromSeq( meta.dataType.asInstanceOf[StructType].names.map { case FILE_PATH => UTF8String.fromString(new File(currentFile.filePath).toString) case FILE_NAME => UTF8String.fromString( - currentFile.filePath.split("/").last) + currentFile.filePath.split(java.io.File.separator).last) case FILE_SIZE => currentFile.fileSize case FILE_MODIFICATION_TIME => currentFile.modificationTime case _ => None // be exhaustive, won't happen @@ -173,15 +155,14 @@ class FileScanRDD( // convert the generic row to an unsafe row val unsafeRowConverter = { - val converter = UnsafeProjection.create( - Array(METADATA_STRUCT)) + val converter = UnsafeProjection.create(Array(meta.dataType)) (row: Row) => { converter(CatalystTypeConverters.convertToCatalyst(row) .asInstanceOf[InternalRow]) } } - metadataStructUnsafeRow = - unsafeRowConverter(Row.fromSeq(Seq(metadataStructGenericRow))) + metadataStructColUnsafeRow = + unsafeRowConverter(Row.fromSeq(Seq(metadataStructColGenericRow))) } } } @@ -191,7 +172,7 @@ class FileScanRDD( */ private def createMetadataStructColumnVector( c: ColumnarBatch, meta: AttributeReference): WritableColumnVector = { - val columnVector = createColumnVector(c.numRows(), METADATA_STRUCT) + val columnVector = new OnHeapColumnVector(c.numRows(), meta.dataType) val filePathBytes = new File(currentFile.filePath).toString.getBytes val fileNameBytes = currentFile.filePath.split("/").last.getBytes var rowId = 0 @@ -228,17 +209,17 @@ class FileScanRDD( * For different row implementations, use different methods to update and append. */ private def addMetadataStructIfNeeded(nextElement: Object): Object = { - if (hasMetadataAttribute) { - val meta = metadataStruct.get + if (metadataStructCol.isDefined) { + val meta = metadataStructCol.get nextElement match { case c: ColumnarBatch => val columnVectorArr = Array.tabulate(c.numCols())(c.column) ++ Array(createMetadataStructColumnVector(c, meta)) new ColumnarBatch(columnVectorArr, c.numRows()) case u: UnsafeRow => - unsafeRowJoiner.asInstanceOf[UnsafeRowJoiner].join(u, metadataStructUnsafeRow) + unsafeRowJoiner.asInstanceOf[UnsafeRowJoiner].join(u, metadataStructColUnsafeRow) case i: InternalRow => - new JoinedRow(i, InternalRow.fromSeq(metadataStructGenericRow.toSeq)) + new JoinedRow(i, InternalRow.fromSeq(metadataStructColGenericRow.toSeq)) } } else { nextElement diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index f43551a0a29e..7efdffd8249a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -212,9 +212,9 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { val outputSchema = readDataColumns.toStructType logInfo(s"Output Data Schema: ${outputSchema.simpleString(5)}") - // outputAttributes should also include referenced metadata columns at the every end + // outputAttributes should also include the referenced metadata struct column at the end val outputAttributes = readDataColumns ++ partitionColumns ++ - plan.references.collect { case MetadataAttribute(attr) => attr } + requiredAttributes.collect { case MetadataAttribute(attr) => attr } val scan = FileSourceScanExec( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 0c558580dc09..291b98fb37c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -75,9 +75,10 @@ case class LogicalRelation( def isOutputColumn(col: AttributeReference): Boolean = { outputNames.exists(name => resolve(col.name, name)) } - // filter out metadata columns that have names conflicting with output columns. if the file - // has a column "_metadata", then the data column should be returned not the metadata column - Seq(FileFormat.FILE_METADATA_COLUMNS).filterNot(isOutputColumn) + // filter out the metadata struct column if it has the name conflicting with output columns. + // if the file has a column "_metadata", + // then the data column should be returned not the metadata struct column + Seq(FileFormat.createFileMetadataCol).filterNot(isOutputColumn) case _ => Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala similarity index 93% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala index 2d6fc8d74fbd..ffa78b09eb67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataColumnsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} -class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { +class FileMetadataStructSuite extends QueryTest with SharedSparkSession { val data0: String = """ @@ -90,7 +90,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { testName: String, fileSchema: StructType) (f: (DataFrame, Map[String, Any], Map[String, Any]) => Unit): Unit = { Seq("csv", "parquet").foreach { testFileFormat => - test(s"metadata columns ($testFileFormat): " + testName) { + test(s"metadata struct ($testFileFormat): " + testName) { withTempDir { dir => // read data0 as CSV and write f0 as testFileFormat val df0 = spark.read.schema(fileSchema).csv( @@ -136,8 +136,8 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { } } - metadataColumnsTest("read partial/all metadata columns", schema) { (df, f0, f1) => - // read all available metadata columns + metadataColumnsTest("read partial/all metadata struct fields", schema) { (df, f0, f1) => + // read all available metadata struct fields checkAnswer( df.select("name", "age", "id", "university", METADATA_FILE_NAME, METADATA_FILE_PATH, @@ -150,7 +150,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) ) - // read a part of metadata columns + // read a part of metadata struct fields checkAnswer( df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), Seq( @@ -160,8 +160,8 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) } - metadataColumnsTest("read metadata columns with random ordering", schema) { (df, f0, f1) => - // read a part of metadata columns with random ordering + metadataColumnsTest("read metadata struct fields with random ordering", schema) { (df, f0, f1) => + // read a part of metadata struct fields with random ordering checkAnswer( df.select(METADATA_FILE_NAME, "name", METADATA_FILE_SIZE, "university"), Seq( @@ -171,7 +171,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) } - metadataColumnsTest("read metadata columns with expressions", schema) { (df, f0, f1) => + metadataColumnsTest("read metadata struct fields with expressions", schema) { (df, f0, f1) => checkAnswer( df.select( // substring of file name @@ -201,7 +201,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) } - metadataColumnsTest("select all will not select metadata columns", schema) { (df, _, _) => + metadataColumnsTest("select all will not select metadata struct fields", schema) { (df, _, _) => checkAnswer( df.select("*"), Seq( @@ -221,19 +221,19 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { Seq( Row("jack", 24, 12345L, "uom", // uom and 12345L will not be overwritten, - // and we still can read metadata columns correctly + // and we still can read metadata struct fields correctly f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), Row("lily", 31, null, "ucb", // ucb and `null` will not be overwritten, - // and we still can read metadata columns correctly + // and we still can read metadata struct fields correctly f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) } - metadataColumnsTest("select only metadata columns", schema) { (df, f0, f1) => + metadataColumnsTest("select only metadata struct fields", schema) { (df, f0, f1) => checkAnswer( df.select(METADATA_FILE_NAME, METADATA_FILE_PATH, METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), @@ -308,7 +308,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { // file schema: name, age, _file_size, _FILE_NAME if (caseSensitive) { // for case sensitive mode: - // _METADATA.FILE_SIZE is not a part of user schema or metadata columns + // _METADATA.FILE_SIZE is not a part of user schema or metadata struct fields val ex = intercept[Exception] { df.select("name", "age", "_METADATA.FILE_SIZE").show() } @@ -316,7 +316,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { // for case sensitive mode: // `_metadata.file_size` and `_metadata.FILE_NAME` are in the user schema - // _metadata.file_name and _metadata.file_modification_time are metadata columns + // _metadata.file_name and _metadata.file_modification_time are metadata struct fields checkAnswer( df.select("name", "age", "`_metadata.file_size`", "`_metadata.FILE_NAME`", "_metadata.file_name", "_metadata.file_modification_time"), @@ -332,13 +332,13 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { // `_metadata.file_size`, `_metadata.FILE_SIZE`, // `_metadata.file_name`, `_metadata.FILE_NAME` are all from the user schema. // different casings of _metadata.file_path and - // _metadata.file_modification_time are metadata columns + // _metadata.file_modification_time are metadata struct fields checkAnswer( df.select("name", "age", // user columns "`_metadata.file_size`", "`_metadata.FILE_SIZE`", "`_metadata.file_name`", "`_metadata.FILE_NAME`", - // metadata columns + // metadata struct fields "_metadata.file_path", "_metadata.FILE_PATH", "_metadata.file_modification_time", "_metadata.FILE_modification_TiMe"), Seq( @@ -359,7 +359,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { withSQLConf("spark.sql.columnVector.offheap.enabled" -> offHeapColumnVectorEnabled) { metadataColumnsTest(s"read metadata with " + s"offheap set to $offHeapColumnVectorEnabled", schema) { (df, f0, f1) => - // read all available metadata columns + // read all available metadata struct fields checkAnswer( df.select("name", "age", "id", "university", METADATA_FILE_NAME, METADATA_FILE_PATH, @@ -372,7 +372,7 @@ class FileMetadataColumnsSuite extends QueryTest with SharedSparkSession { ) ) - // read a part of metadata columns + // read a part of metadata struct fields checkAnswer( df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), Seq( From f78fe9241035d84f93a54233ba2be450313f8211 Mon Sep 17 00:00:00 2001 From: yaohua Date: Wed, 24 Nov 2021 17:09:56 -0800 Subject: [PATCH 10/21] simplify row cache --- .../execution/datasources/FileScanRDD.scala | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 4b9b1225fb62..6969b152cbed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -24,8 +24,8 @@ import scala.util.control.NonFatal import org.apache.spark.{Partition => RDDPartition, SparkUpgradeException, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeRowJoiner, UnsafeRowJoiner} import org.apache.spark.sql.errors.QueryExecutionErrors @@ -121,8 +121,8 @@ class FileScanRDD( // metadata struct unsafe row, will only be updated when the current file is changed @volatile private var metadataStructColUnsafeRow: UnsafeRow = _ - // metadata generic row, will only be updated when the current file is changed - @volatile private var metadataStructColGenericRow: Row = _ + // metadata internal row, will only be updated when the current file is changed + @volatile private var metadataStructColInternalRow: InternalRow = _ // an unsafe joiner to join an unsafe row with the metadata unsafe row lazy private val unsafeRowJoiner = if (metadataStructCol.isDefined) { @@ -138,11 +138,11 @@ class FileScanRDD( val meta = metadataStructCol.get if (currentFile == null) { metadataStructColUnsafeRow = null - metadataStructColGenericRow = null + metadataStructColInternalRow = null } else { // make an generic row assert(meta.dataType.isInstanceOf[StructType]) - metadataStructColGenericRow = Row.fromSeq( + metadataStructColInternalRow = InternalRow.fromSeq( meta.dataType.asInstanceOf[StructType].names.map { case FILE_PATH => UTF8String.fromString(new File(currentFile.filePath).toString) case FILE_NAME => UTF8String.fromString( @@ -156,13 +156,10 @@ class FileScanRDD( // convert the generic row to an unsafe row val unsafeRowConverter = { val converter = UnsafeProjection.create(Array(meta.dataType)) - (row: Row) => { - converter(CatalystTypeConverters.convertToCatalyst(row) - .asInstanceOf[InternalRow]) - } + (row: InternalRow) => converter(row) } metadataStructColUnsafeRow = - unsafeRowConverter(Row.fromSeq(Seq(metadataStructColGenericRow))) + unsafeRowConverter(InternalRow.fromSeq(Seq(metadataStructColInternalRow))) } } } @@ -218,8 +215,7 @@ class FileScanRDD( new ColumnarBatch(columnVectorArr, c.numRows()) case u: UnsafeRow => unsafeRowJoiner.asInstanceOf[UnsafeRowJoiner].join(u, metadataStructColUnsafeRow) - case i: InternalRow => - new JoinedRow(i, InternalRow.fromSeq(metadataStructColGenericRow.toSeq)) + case i: InternalRow => new JoinedRow(i, metadataStructColInternalRow) } } else { nextElement From 2baccdbaac45f3d1178a917eaa81eee8c00e7d5b Mon Sep 17 00:00:00 2001 From: yaohua Date: Wed, 24 Nov 2021 17:24:49 -0800 Subject: [PATCH 11/21] add todo --- .../org/apache/spark/sql/execution/DataSourceScanExec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index cbb37857514f..2b3a8e219f14 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -368,6 +368,7 @@ case class FileSourceScanExec( @transient private lazy val pushedDownFilters = { val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) + // TODO: should be able to push filters containing metadata struct down to skip files dataFilters .filterNot( _.references.exists { From 8b8b9fa5da89dd60c7f9dd88de3c117bf32b6cdb Mon Sep 17 00:00:00 2001 From: yaohua Date: Thu, 25 Nov 2021 19:54:35 -0800 Subject: [PATCH 12/21] comments and refactor test to use json --- .../execution/datasources/FileScanRDD.scala | 15 +- .../execution/datasources/SchemaPruning.scala | 7 +- .../datasources/FileMetadataStructSuite.scala | 305 ++++++------------ 3 files changed, 120 insertions(+), 207 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 6969b152cbed..d72e19a45e64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.execution.datasources -import java.io.{Closeable, File, FileNotFoundException, IOException} +import java.io.{Closeable, FileNotFoundException, IOException} import scala.util.control.NonFatal +import org.apache.hadoop.fs.Path + import org.apache.spark.{Partition => RDDPartition, SparkUpgradeException, TaskContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} @@ -141,12 +143,12 @@ class FileScanRDD( metadataStructColInternalRow = null } else { // make an generic row + val path = new Path(currentFile.filePath) assert(meta.dataType.isInstanceOf[StructType]) metadataStructColInternalRow = InternalRow.fromSeq( meta.dataType.asInstanceOf[StructType].names.map { - case FILE_PATH => UTF8String.fromString(new File(currentFile.filePath).toString) - case FILE_NAME => UTF8String.fromString( - currentFile.filePath.split(java.io.File.separator).last) + case FILE_PATH => UTF8String.fromString(path.toString) + case FILE_NAME => UTF8String.fromString(path.getName) case FILE_SIZE => currentFile.fileSize case FILE_MODIFICATION_TIME => currentFile.modificationTime case _ => None // be exhaustive, won't happen @@ -170,8 +172,9 @@ class FileScanRDD( private def createMetadataStructColumnVector( c: ColumnarBatch, meta: AttributeReference): WritableColumnVector = { val columnVector = new OnHeapColumnVector(c.numRows(), meta.dataType) - val filePathBytes = new File(currentFile.filePath).toString.getBytes - val fileNameBytes = currentFile.filePath.split("/").last.getBytes + val path = new Path(currentFile.filePath) + val filePathBytes = path.toString.getBytes + val fileNameBytes = path.getName.getBytes var rowId = 0 assert(meta.dataType.isInstanceOf[StructType]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala index 4f331c7bf487..93bd1acc7377 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala @@ -164,7 +164,12 @@ object SchemaPruning extends Rule[LogicalPlan] { outputRelation: LogicalRelation, prunedBaseRelation: HadoopFsRelation) = { val prunedOutput = getPrunedOutput(outputRelation.output, prunedBaseRelation.schema) - outputRelation.copy(relation = prunedBaseRelation, output = prunedOutput) + // also add the metadata output if any + // TODO: should be able to prune the metadata schema + val metaOutput = outputRelation.output.collect { + case MetadataAttribute(attr) => attr + } + outputRelation.copy(relation = prunedBaseRelation, output = prunedOutput ++ metaOutput) } // Prune the given output to make it consistent with `requiredSchema`. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala index ffa78b09eb67..8b4b3d22920e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources import java.io.File -import java.nio.file.Files import java.text.SimpleDateFormat import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} @@ -28,30 +27,23 @@ import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructFiel class FileMetadataStructSuite extends QueryTest with SharedSparkSession { - val data0: String = - """ - |jack,24,12345,uom - |""".stripMargin + val data0: Seq[Row] = Seq(Row("jack", 24, Row(12345L, "uom"))) - val data1: String = - """ - |lily,31,,ucb - |""".stripMargin + val data1: Seq[Row] = Seq(Row("lily", 31, Row(54321L, "ucb"))) val schema: StructType = new StructType() .add(StructField("name", StringType)) .add(StructField("age", IntegerType)) - .add(StructField("id", LongType)) - .add(StructField("university", StringType)) + .add(StructField("info", new StructType() + .add(StructField("id", LongType)) + .add(StructField("university", StringType)))) - // make sure: - // `_metadata.file_size` (a flat column) and _metadata.file_size (a field in a strut) - // are not the same columns, users could read both correctly val schemaWithNameConflicts: StructType = new StructType() .add(StructField("name", StringType)) .add(StructField("age", IntegerType)) - .add(StructField("_metadata.file_size", LongType)) - .add(StructField("_metadata.FILE_NAME", StringType)) + .add(StructField("_METADATA", new StructType() + .add(StructField("id", LongType)) + .add(StructField("university", StringType)))) private val METADATA_FILE_PATH = "_metadata.file_path" @@ -62,25 +54,15 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { private val METADATA_FILE_MODIFICATION_TIME = "_metadata.file_modification_time" /** - * Create a CSV file named `fileName` with `data` under `dir` directory. - */ - private def createCSVFile(data: String, dir: File, fileName: String): String = { - val dataFile = new File(dir, s"/$fileName") - dataFile.getParentFile.mkdirs() - val bytes = data.getBytes() - Files.write(dataFile.toPath, bytes) - dataFile.getCanonicalPath - } - - /** - * This test wrapper will test for both row-based and column-based file formats (csv and parquet) - * 1. read data0 and data1 and write them as testFileFormat: f0 and f1 - * 2. read both f0 and f1, return the df to the downstream for further testing - * 3. construct actual metadata map for both f0 and f1 to the downstream for further testing + * This test wrapper will test for both row-based and column-based file formats: + * (json and parquet) with nested schema: + * 1. create df0 and df1 and save them as testFileFormat under /data/f0 and /data/f1 + * 2. read the path /data, return the df for further testing + * 3. create actual metadata maps for both files under /data/f0 and /data/f1 for further testing * * The final df will have data: - * jack | 24 | 12345 | uom - * lily | 31 | null | ucb + * jack | 24 | {12345, uom} + * lily | 31 | {54321, ucb} * * The schema of the df will be the `fileSchema` provided to this method * @@ -89,24 +71,21 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { private def metadataColumnsTest( testName: String, fileSchema: StructType) (f: (DataFrame, Map[String, Any], Map[String, Any]) => Unit): Unit = { - Seq("csv", "parquet").foreach { testFileFormat => + Seq("json", "parquet").foreach { testFileFormat => test(s"metadata struct ($testFileFormat): " + testName) { withTempDir { dir => - // read data0 as CSV and write f0 as testFileFormat - val df0 = spark.read.schema(fileSchema).csv( - createCSVFile(data0, dir, "temp/0") - ) - val f0Path = new File(dir, "data/f0").getCanonicalPath - df0.coalesce(1).write.format(testFileFormat).save(f0Path) + import scala.collection.JavaConverters._ - // read data1 as CSV and write f1 as testFileFormat - val df1 = spark.read.schema(fileSchema).csv( - createCSVFile(data1, dir, "temp/1") - ) - val f1Path = new File(dir, "data/f1").getCanonicalPath - df1.coalesce(1).write.format(testFileFormat).save(f1Path) + // 1. create df0 and df1 and save under /data/f0 and /data/f1 + val df0 = spark.createDataFrame(data0.asJava, fileSchema) + val f0 = new File(dir, "data/f0").getCanonicalPath + df0.coalesce(1).write.format(testFileFormat).save(f0) + + val df1 = spark.createDataFrame(data1.asJava, fileSchema) + val f1 = new File(dir, "data/f1").getCanonicalPath + df1.coalesce(1).write.format(testFileFormat).save(f1) - // read both f0 and f1 + // 2. read both f0 and f1 val df = spark.read.format(testFileFormat).schema(fileSchema) .load(new File(dir, "data").getCanonicalPath + "/*") @@ -116,7 +95,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { val realF1 = new File(dir, "data/f1").listFiles() .filter(_.getName.endsWith(s".$testFileFormat")).head - // construct f0 and f1 metadata data + // 3. create f0 and f1 metadata data val f0Metadata = Map( METADATA_FILE_PATH -> realF0.toURI.toString, METADATA_FILE_NAME -> realF0.getName, @@ -139,20 +118,22 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { metadataColumnsTest("read partial/all metadata struct fields", schema) { (df, f0, f1) => // read all available metadata struct fields checkAnswer( - df.select("name", "age", "id", "university", + df.select("name", "age", "info", METADATA_FILE_NAME, METADATA_FILE_PATH, METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), Seq( - Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + Row("jack", 24, Row(12345L, "uom"), + f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), - Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + Row("lily", 31, Row(54321L, "ucb"), + f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) // read a part of metadata struct fields checkAnswer( - df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), + df.select("name", "info.university", METADATA_FILE_NAME, METADATA_FILE_SIZE), Seq( Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) @@ -163,7 +144,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { metadataColumnsTest("read metadata struct fields with random ordering", schema) { (df, f0, f1) => // read a part of metadata struct fields with random ordering checkAnswer( - df.select(METADATA_FILE_NAME, "name", METADATA_FILE_SIZE, "university"), + df.select(METADATA_FILE_NAME, "name", METADATA_FILE_SIZE, "info.university"), Seq( Row(f0(METADATA_FILE_NAME), "jack", f0(METADATA_FILE_SIZE), "uom"), Row(f1(METADATA_FILE_NAME), "lily", f1(METADATA_FILE_SIZE), "ucb") @@ -205,35 +186,33 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { checkAnswer( df.select("*"), Seq( - Row("jack", 24, 12345L, "uom"), - Row("lily", 31, null, "ucb") + Row("jack", 24, Row(12345L, "uom")), + Row("lily", 31, Row(54321L, "ucb")) ) ) } - metadataColumnsTest("metadata fields and `_metadata.file_size` " + - "are not the same", schemaWithNameConflicts) { (df, f0, f1) => - // here: the data has the schema: name, age, _metadata.file_size, _metadata.file_name + metadataColumnsTest("metadata will not overwrite user data", + schemaWithNameConflicts) { (df, _, _) => + // the user data has the schema: name, age, _metadata.id, _metadata.university + + // select user data checkAnswer( - df.select("name", "age", "`_metadata.file_size`", "`_metadata.FILE_NAME`", - METADATA_FILE_NAME, METADATA_FILE_PATH, - METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), + df.select("name", "age", "_METADATA", "_metadata"), Seq( - Row("jack", 24, 12345L, "uom", - // uom and 12345L will not be overwritten, - // and we still can read metadata struct fields correctly - f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), - f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), - Row("lily", 31, null, "ucb", - // ucb and `null` will not be overwritten, - // and we still can read metadata struct fields correctly - f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), - f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) + Row("jack", 24, Row(12345L, "uom"), Row(12345L, "uom")), + Row("lily", 31, Row(54321L, "ucb"), Row(54321L, "ucb")) ) ) + + // select metadata will fail when analysis + val ex = intercept[AnalysisException] { + df.select("name", METADATA_FILE_NAME).collect() + } + assert(ex.getMessage.contains("No such struct field file_name in id, university")) } - metadataColumnsTest("select only metadata struct fields", schema) { (df, f0, f1) => + metadataColumnsTest("select only metadata", schema) { (df, f0, f1) => checkAnswer( df.select(METADATA_FILE_NAME, METADATA_FILE_PATH, METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), @@ -244,13 +223,20 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) + checkAnswer( + df.select("_metadata"), + Seq( + Row(Row(f0(METADATA_FILE_PATH), f0(METADATA_FILE_NAME), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME))), + Row(Row(f1(METADATA_FILE_PATH), f1(METADATA_FILE_NAME), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME))) + ) + ) } metadataColumnsTest("select and re-select", schema) { (df, f0, f1) => - // test and make sure we are not accidentally making unsafe row - // to the more general internal row, thus it will fail to re-select checkAnswer( - df.select("name", "age", "id", "university", + df.select("name", "age", "info", METADATA_FILE_NAME, METADATA_FILE_PATH, METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME) .select("name", "file_path"), // cast _metadata.file_path as file_path @@ -304,52 +290,49 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { metadataColumnsTest(s"upper/lower case when case " + s"sensitive is $caseSensitive", schemaWithNameConflicts) { (df, f0, f1) => withSQLConf("spark.sql.caseSensitive" -> caseSensitive.toString) { + // file schema: name, age, _METADATA.id, _METADATA.university - // file schema: name, age, _file_size, _FILE_NAME if (caseSensitive) { // for case sensitive mode: - // _METADATA.FILE_SIZE is not a part of user schema or metadata struct fields - val ex = intercept[Exception] { - df.select("name", "age", "_METADATA.FILE_SIZE").show() - } - assert(ex.getMessage.contains("_METADATA.FILE_SIZE")) - - // for case sensitive mode: - // `_metadata.file_size` and `_metadata.FILE_NAME` are in the user schema - // _metadata.file_name and _metadata.file_modification_time are metadata struct fields + // _METADATA is user data + // _metadata is metadata checkAnswer( - df.select("name", "age", "`_metadata.file_size`", "`_metadata.FILE_NAME`", - "_metadata.file_name", "_metadata.file_modification_time"), + df.select("name", "age", "_METADATA", "_metadata"), Seq( - Row("jack", 24, 12345L, "uom", - f0(METADATA_FILE_NAME), f0(METADATA_FILE_MODIFICATION_TIME)), - Row("lily", 31, null, "ucb", - f1(METADATA_FILE_NAME), f1(METADATA_FILE_MODIFICATION_TIME)) + Row("jack", 24, Row(12345L, "uom"), + Row(f0(METADATA_FILE_PATH), f0(METADATA_FILE_NAME), + f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME))), + Row("lily", 31, Row(54321L, "ucb"), + Row(f1(METADATA_FILE_PATH), f1(METADATA_FILE_NAME), + f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME))) ) ) } else { // for case insensitive mode: - // `_metadata.file_size`, `_metadata.FILE_SIZE`, - // `_metadata.file_name`, `_metadata.FILE_NAME` are all from the user schema. - // different casings of _metadata.file_path and - // _metadata.file_modification_time are metadata struct fields + // _METADATA and _metadata are both user data + + // select user data checkAnswer( df.select("name", "age", // user columns - "`_metadata.file_size`", "`_metadata.FILE_SIZE`", - "`_metadata.file_name`", "`_metadata.FILE_NAME`", - // metadata struct fields - "_metadata.file_path", "_metadata.FILE_PATH", - "_metadata.file_modification_time", "_metadata.FILE_modification_TiMe"), + "_METADATA", "_metadata", + "_metadata.ID", "_METADATA.UniVerSity"), Seq( - Row("jack", 24, 12345L, 12345L, "uom", "uom", - f0(METADATA_FILE_PATH), f0(METADATA_FILE_PATH), - f0(METADATA_FILE_MODIFICATION_TIME), f0(METADATA_FILE_MODIFICATION_TIME)), - Row("lily", 31, null, null, "ucb", "ucb", - f1(METADATA_FILE_PATH), f1(METADATA_FILE_PATH), - f1(METADATA_FILE_MODIFICATION_TIME), f1(METADATA_FILE_MODIFICATION_TIME)) + Row("jack", 24, Row(12345L, "uom"), Row(12345L, "uom"), 12345L, "uom"), + Row("lily", 31, Row(54321L, "ucb"), Row(54321L, "ucb"), 54321L, "ucb") ) ) + + // select metadata will fail when analysis - metadata cannot overwrite user data + val ex = intercept[AnalysisException] { + df.select("name", "_metadata.file_name").collect() + } + assert(ex.getMessage.contains("No such struct field file_name in id, university")) + + val ex1 = intercept[AnalysisException] { + df.select("name", "_METADATA.file_NAME").collect() + } + assert(ex1.getMessage.contains("No such struct field file_NAME in id, university")) } } } @@ -361,20 +344,20 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { s"offheap set to $offHeapColumnVectorEnabled", schema) { (df, f0, f1) => // read all available metadata struct fields checkAnswer( - df.select("name", "age", "id", "university", + df.select("name", "age", "info", METADATA_FILE_NAME, METADATA_FILE_PATH, METADATA_FILE_SIZE, METADATA_FILE_MODIFICATION_TIME), Seq( - Row("jack", 24, 12345L, "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), + Row("jack", 24, Row(12345L, "uom"), f0(METADATA_FILE_NAME), f0(METADATA_FILE_PATH), f0(METADATA_FILE_SIZE), f0(METADATA_FILE_MODIFICATION_TIME)), - Row("lily", 31, null, "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), + Row("lily", 31, Row(54321L, "ucb"), f1(METADATA_FILE_NAME), f1(METADATA_FILE_PATH), f1(METADATA_FILE_SIZE), f1(METADATA_FILE_MODIFICATION_TIME)) ) ) // read a part of metadata struct fields checkAnswer( - df.select("name", "university", METADATA_FILE_NAME, METADATA_FILE_SIZE), + df.select("name", "info.university", METADATA_FILE_NAME, METADATA_FILE_SIZE), Seq( Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) @@ -384,97 +367,19 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { } } - ////////////////////////// - // TEST METADATA STRUCT // - ////////////////////////// - - // has _metadata.file_name - val jsonData0 = - """ - |{ - | "name":"jack", - | "_metadata":{ - | "age":24, - | "file_name":"jack.json" - | } - |} - |""".stripMargin - val jsonSchema0: StructType = new StructType() - .add(StructField("name", StringType)) - .add(StructField("_metadata", new StructType() - .add(StructField("age", IntegerType)) - .add(StructField("file_name", StringType)))) - - // no naming conflicts at all - val jsonData1 = - """ - |{ - | "name":"jack", - | "metadata":{ - | "age":24, - | "file_name":"jack.json" - | } - |} - |""".stripMargin - val jsonSchema1: StructType = new StructType() - .add(StructField("name", StringType)) - .add(StructField("metadata", new StructType() - .add(StructField("age", IntegerType)) - .add(StructField("file_name", StringType)))) - - /** - * Create a JSON file named `fileName` with `data` under `dir` directory. - */ - private def createJSONFile(data: String, dir: File, fileName: String): String = { - val dataFile = new File(dir, s"/$fileName") - dataFile.getParentFile.mkdirs() - val bytes = data.filter(_ >= ' ').getBytes - Files.write(dataFile.toPath, bytes) - dataFile.getCanonicalPath - } - - test("test data schema has _metadata struct") { - withTempDir { dir => - - // 0 - select metadata will fail when analysis - val df0 = spark.read.schema(jsonSchema0).json( - createJSONFile(jsonData0, dir, "temp/0")) - checkAnswer( - df0.select("name", "_metadata.file_name"), - Row("jack", "jack.json") - ) - val ex0 = intercept[AnalysisException] { - df0.select("name", "_metadata.file_path").show() - } - assert(ex0.getMessage.contains("No such struct field file_path in age, file_name")) - - // 1 - no conflict, everything is fine - val df1 = spark.read.schema(jsonSchema1).json( - createJSONFile(jsonData1, dir, "temp/1")) - - // get metadata - val f1 = new File(dir, "temp/1") - val metadata = Map( - METADATA_FILE_PATH -> f1.toURI.toString, - METADATA_FILE_NAME -> f1.getName, - METADATA_FILE_SIZE -> f1.length(), - METADATA_FILE_MODIFICATION_TIME -> f1.lastModified() - ) - - checkAnswer( - df1.select("name", "metadata.file_name", - "_metadata.file_path", "_metadata.file_name", - "_metadata.file_size", "_metadata.file_modification_time", - "_metadata"), - Row("jack", "jack.json", - metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME), - metadata(METADATA_FILE_SIZE), metadata(METADATA_FILE_MODIFICATION_TIME), - // struct of _metadata - Row( - metadata(METADATA_FILE_PATH), metadata(METADATA_FILE_NAME), - metadata(METADATA_FILE_SIZE), metadata(METADATA_FILE_MODIFICATION_TIME)) + Seq("true", "false").foreach { enabled => + withSQLConf("spark.sql.optimizer.nestedSchemaPruning.enabled" -> enabled) { + metadataColumnsTest(s"read metadata with" + + s"nestedSchemaPruning set to $enabled", schema) { (df, f0, f1) => + // read a part of data: schema pruning + checkAnswer( + df.select("name", "info.university", METADATA_FILE_NAME, METADATA_FILE_SIZE), + Seq( + Row("jack", "uom", f0(METADATA_FILE_NAME), f0(METADATA_FILE_SIZE)), + Row("lily", "ucb", f1(METADATA_FILE_NAME), f1(METADATA_FILE_SIZE)) + ) ) - ) + } } } } From d984b505601729802971f710a8f168c6b41031c6 Mon Sep 17 00:00:00 2001 From: yaohua Date: Sun, 19 Dec 2021 16:38:55 +0800 Subject: [PATCH 13/21] flat and struct --- .../sql/execution/DataSourceScanExec.scala | 20 +-- .../execution/datasources/FileScanRDD.scala | 141 +++++++++--------- .../datasources/FileSourceStrategy.scala | 35 ++++- 3 files changed, 104 insertions(+), 92 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 2b3a8e219f14..8dda591fcab8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -195,17 +195,13 @@ case class FileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { - lazy val metadataStructCol: Option[AttributeReference] = - output.collectFirst { case MetadataAttribute(attr) => attr } + lazy val metadataColumns: Seq[AttributeReference] = + output.collect { case MetadataAttribute(attr) => attr } // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. override lazy val supportsColumnar: Boolean = { - // schema without the file metadata struct column - val fileSchema = if (metadataStructCol.isEmpty) schema else { - output.filter(_.exprId != metadataStructCol.get.exprId).toStructType - } - relation.fileFormat.supportBatch(relation.sparkSession, fileSchema) + relation.fileFormat.supportBatch(relation.sparkSession, schema) } private lazy val needsUnsafeRowConversion: Boolean = { @@ -222,9 +218,7 @@ case class FileSourceScanExec( partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf).map { vectorTypes => // for column-based file format, append metadata struct column's vector type classes if any - vectorTypes ++ (if (metadataStructCol.isDefined) { - Seq(classOf[OnHeapColumnVector].getName) - } else Seq.empty) + vectorTypes ++ Seq.fill(metadataColumns.size)(classOf[OnHeapColumnVector].getName) } private lazy val driverMetrics: HashMap[String, Long] = HashMap.empty @@ -368,7 +362,7 @@ case class FileSourceScanExec( @transient private lazy val pushedDownFilters = { val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) - // TODO: should be able to push filters containing metadata struct down to skip files + // TODO: should be able to push filters containing metadata columns down to skip files dataFilters .filterNot( _.references.exists { @@ -619,7 +613,7 @@ case class FileSourceScanExec( } new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions, - requiredSchema, metadataStructCol) + requiredSchema, metadataColumns) } /** @@ -676,7 +670,7 @@ case class FileSourceScanExec( FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions, - requiredSchema, metadataStructCol) + requiredSchema, metadataColumns) } // Filters unused DynamicPruningExpression expressions - one which has been replaced diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index d72e19a45e64..7f6167e401ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -29,11 +29,11 @@ import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeRowJoiner, UnsafeRowJoiner} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.FileFormat._ import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.NextIterator @@ -70,7 +70,7 @@ class FileScanRDD( readFunction: (PartitionedFile) => Iterator[InternalRow], @transient val filePartitions: Seq[FilePartition], val requiredSchema: StructType = StructType(Seq.empty), - val metadataStructCol: Option[AttributeReference] = None) + val metadataColumns: Seq[AttributeReference] = Seq.empty) extends RDD[InternalRow](sparkSession.sparkContext, Nil) { private val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles @@ -121,104 +121,97 @@ class FileScanRDD( // FILE METADATA METHODS // /////////////////////////// - // metadata struct unsafe row, will only be updated when the current file is changed - @volatile private var metadataStructColUnsafeRow: UnsafeRow = _ - // metadata internal row, will only be updated when the current file is changed - @volatile private var metadataStructColInternalRow: InternalRow = _ + // metadata columns unsafe row, will only be updated when the current file is changed + @volatile private var metadataColumnsUnsafeRow: UnsafeRow = _ + // metadata columns internal row, will only be updated when the current file is changed + @volatile private var metadataColumnsInternalRow: InternalRow = _ // an unsafe joiner to join an unsafe row with the metadata unsafe row - lazy private val unsafeRowJoiner = - if (metadataStructCol.isDefined) { - GenerateUnsafeRowJoiner.create(requiredSchema, Seq(metadataStructCol.get).toStructType) - } + lazy private val metadataUnsafeRowJoiner = + GenerateUnsafeRowJoiner.create(requiredSchema, metadataColumns.toStructType) + // metadata columns unsafe row converter + lazy private val unsafeRowConverter = { + val metadataColumnsDataTypes = metadataColumns.map(_.dataType).toArray + val converter = UnsafeProjection.create(metadataColumnsDataTypes) + (row: InternalRow) => converter(row) + } /** - * For each partitioned file, metadata struct for each record in the file are exactly same. - * Only update metadata struct when `currentFile` is changed. + * For each partitioned file, metadata columns for each record in the file are exactly same. + * Only update metadata columns when `currentFile` is changed. */ - private def updateMetadataStruct(): Unit = { - if (metadataStructCol.isDefined) { - val meta = metadataStructCol.get + private def updateMetadataColumns(): Unit = { + if (metadataColumns.nonEmpty) { if (currentFile == null) { - metadataStructColUnsafeRow = null - metadataStructColInternalRow = null + metadataColumnsUnsafeRow = null + metadataColumnsInternalRow = null } else { - // make an generic row + // construct an internal row val path = new Path(currentFile.filePath) - assert(meta.dataType.isInstanceOf[StructType]) - metadataStructColInternalRow = InternalRow.fromSeq( - meta.dataType.asInstanceOf[StructType].names.map { + metadataColumnsInternalRow = InternalRow.fromSeq( + metadataColumns.map(_.name).map { case FILE_PATH => UTF8String.fromString(path.toString) case FILE_NAME => UTF8String.fromString(path.getName) case FILE_SIZE => currentFile.fileSize case FILE_MODIFICATION_TIME => currentFile.modificationTime - case _ => None // be exhaustive, won't happen } ) - - // convert the generic row to an unsafe row - val unsafeRowConverter = { - val converter = UnsafeProjection.create(Array(meta.dataType)) - (row: InternalRow) => converter(row) - } - metadataStructColUnsafeRow = - unsafeRowConverter(InternalRow.fromSeq(Seq(metadataStructColInternalRow))) + // convert the internal row to an unsafe row + metadataColumnsUnsafeRow = unsafeRowConverter(metadataColumnsInternalRow) } } } /** - * Create a writable column vector containing all required metadata fields + * Create a writable column vector containing all required metadata columns */ - private def createMetadataStructColumnVector( - c: ColumnarBatch, meta: AttributeReference): WritableColumnVector = { - val columnVector = new OnHeapColumnVector(c.numRows(), meta.dataType) + private def createMetadataColumnVector(c: ColumnarBatch): Array[WritableColumnVector] = { val path = new Path(currentFile.filePath) val filePathBytes = path.toString.getBytes val fileNameBytes = path.getName.getBytes var rowId = 0 - - assert(meta.dataType.isInstanceOf[StructType]) - meta.dataType.asInstanceOf[StructType].names.zipWithIndex.foreach { case (name, ind) => - name match { - case FILE_PATH => - rowId = 0 - // use a tight-loop for better performance - while (rowId < c.numRows()) { - columnVector.getChild(ind).putByteArray(rowId, filePathBytes) - rowId += 1 - } - case FILE_NAME => - rowId = 0 - // use a tight-loop for better performance - while (rowId < c.numRows()) { - columnVector.getChild(ind).putByteArray(rowId, fileNameBytes) - rowId += 1 - } - case FILE_SIZE => - columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.fileSize) - case FILE_MODIFICATION_TIME => - columnVector.getChild(ind).putLongs(0, c.numRows(), currentFile.modificationTime) - case _ => // be exhaustive, won't happen - } - } - columnVector + metadataColumns.map(_.name).map { + case FILE_PATH => + val columnVector = new OnHeapColumnVector(c.numRows(), StringType) + rowId = 0 + // use a tight-loop for better performance + while (rowId < c.numRows()) { + columnVector.putByteArray(rowId, filePathBytes) + rowId += 1 + } + columnVector + case FILE_NAME => + val columnVector = new OnHeapColumnVector(c.numRows(), StringType) + rowId = 0 + // use a tight-loop for better performance + while (rowId < c.numRows()) { + columnVector.putByteArray(rowId, fileNameBytes) + rowId += 1 + } + columnVector + case FILE_SIZE => + val columnVector = new OnHeapColumnVector(c.numRows(), LongType) + columnVector.putLongs(0, c.numRows(), currentFile.fileSize) + columnVector + case FILE_MODIFICATION_TIME => + val columnVector = new OnHeapColumnVector(c.numRows(), LongType) + columnVector.putLongs(0, c.numRows(), currentFile.modificationTime) + columnVector + }.toArray } /** - * Add metadata struct at the end of nextElement if needed. + * Add metadata columns at the end of nextElement if needed. * For different row implementations, use different methods to update and append. */ - private def addMetadataStructIfNeeded(nextElement: Object): Object = { - if (metadataStructCol.isDefined) { - val meta = metadataStructCol.get + private def addMetadataColumnsIfNeeded(nextElement: Object): Object = { + if (metadataColumns.nonEmpty) { nextElement match { case c: ColumnarBatch => - val columnVectorArr = Array.tabulate(c.numCols())(c.column) ++ - Array(createMetadataStructColumnVector(c, meta)) - new ColumnarBatch(columnVectorArr, c.numRows()) - case u: UnsafeRow => - unsafeRowJoiner.asInstanceOf[UnsafeRowJoiner].join(u, metadataStructColUnsafeRow) - case i: InternalRow => new JoinedRow(i, metadataStructColInternalRow) + new ColumnarBatch( + Array.tabulate(c.numCols())(c.column) ++ createMetadataColumnVector(c), + c.numRows()) + case u: UnsafeRow => metadataUnsafeRowJoiner.join(u, metadataColumnsUnsafeRow) + case i: InternalRow => new JoinedRow(i, metadataColumnsInternalRow) } } else { nextElement @@ -241,7 +234,7 @@ class FileScanRDD( } inputMetrics.incRecordsRead(1) } - addMetadataStructIfNeeded(nextElement) + addMetadataColumnsIfNeeded(nextElement) } private def readCurrentFile(): Iterator[InternalRow] = { @@ -257,7 +250,7 @@ class FileScanRDD( private def nextIterator(): Boolean = { if (files.hasNext) { currentFile = files.next() - updateMetadataStruct() + updateMetadataColumns() logInfo(s"Reading File $currentFile") // Sets InputFileBlockHolder for the file block's information InputFileBlockHolder.set(currentFile.filePath, currentFile.start, currentFile.length) @@ -325,7 +318,7 @@ class FileScanRDD( } } else { currentFile = null - updateMetadataStruct() + updateMetadataColumns() InputFileBlockHolder.unset() false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 7efdffd8249a..052c98582c8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ScanOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} -import org.apache.spark.sql.types.{DoubleType, FloatType} +import org.apache.spark.sql.execution.datasources.FileFormat.METADATA_NAME +import org.apache.spark.sql.types.{DoubleType, FloatType, StructType} import org.apache.spark.util.collection.BitSet /** @@ -212,9 +213,19 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { val outputSchema = readDataColumns.toStructType logInfo(s"Output Data Schema: ${outputSchema.simpleString(5)}") - // outputAttributes should also include the referenced metadata struct column at the end - val outputAttributes = readDataColumns ++ partitionColumns ++ - requiredAttributes.collect { case MetadataAttribute(attr) => attr } + val metadataStructOpt = requiredAttributes.collectFirst { + case MetadataAttribute(attr) => attr + } + + // TODO (yaohua): should be able to prune the metadata struct only containing what needed + val metadataColumns = metadataStructOpt.map { metadataStruct => + metadataStruct.dataType.asInstanceOf[StructType].fields.map { field => + MetadataAttribute(field.name, field.dataType) + }.toSeq + }.getOrElse(Seq.empty) + + // outputAttributes should also include the metadata columns at the very end + val outputAttributes = readDataColumns ++ partitionColumns ++ metadataColumns val scan = FileSourceScanExec( @@ -227,8 +238,22 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { dataFilters, table.map(_.identifier)) + val metadataAliasOpt = metadataStructOpt.map { metadataStruct => + Alias(CreateStruct(metadataColumns), METADATA_NAME)(exprId = metadataStruct.exprId) + } + + // extra Project node: wrap flat metadata columns to a metadata struct + val withMetadataProjections = metadataStructOpt.map { metadataStruct => + val metadataAlias = + Alias(CreateStruct(metadataColumns), METADATA_NAME)(exprId = metadataStruct.exprId) + execution.ProjectExec( + scan.output.filterNot(metadataColumns.contains) :+ metadataAlias, scan) + }.getOrElse(scan) + val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And) - val withFilter = afterScanFilter.map(execution.FilterExec(_, scan)).getOrElse(scan) + val withFilter = afterScanFilter + .map(execution.FilterExec(_, withMetadataProjections)) + .getOrElse(withMetadataProjections) val withProjections = if (projects == withFilter.output) { withFilter } else { From 0f6eccd4c0aea1578c262cc641318a51468bcbe8 Mon Sep 17 00:00:00 2001 From: yaohua Date: Mon, 20 Dec 2021 08:39:11 +0800 Subject: [PATCH 14/21] nit remove --- .../spark/sql/execution/datasources/FileSourceStrategy.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 052c98582c8b..3b677fc2641a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -238,10 +238,6 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { dataFilters, table.map(_.identifier)) - val metadataAliasOpt = metadataStructOpt.map { metadataStruct => - Alias(CreateStruct(metadataColumns), METADATA_NAME)(exprId = metadataStruct.exprId) - } - // extra Project node: wrap flat metadata columns to a metadata struct val withMetadataProjections = metadataStructOpt.map { metadataStruct => val metadataAlias = From f780bf2c9b8b2ca140dcd3da66f9d2df0547a1f5 Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 08:39:00 +0800 Subject: [PATCH 15/21] comments --- .../spark/sql/execution/DataSourceScanExec.scala | 12 ++++-------- .../execution/datasources/FileSourceStrategy.scala | 2 +- .../datasources/FileMetadataStructSuite.scala | 3 ++- 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 8dda591fcab8..e3212e3d715e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -363,14 +363,10 @@ case class FileSourceScanExec( private lazy val pushedDownFilters = { val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation) // TODO: should be able to push filters containing metadata columns down to skip files - dataFilters - .filterNot( - _.references.exists { - case MetadataAttribute(_) => true - case _ => false - } - ) - .flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) + dataFilters.filterNot(_.references.exists { + case MetadataAttribute(_) => true + case _ => false + }).flatMap(DataSourceStrategy.translateFilter(_, supportNestedPredicatePushdown)) } override lazy val metadata: Map[String, String] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 3b677fc2641a..c1282fa69ca8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -243,7 +243,7 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging { val metadataAlias = Alias(CreateStruct(metadataColumns), METADATA_NAME)(exprId = metadataStruct.exprId) execution.ProjectExec( - scan.output.filterNot(metadataColumns.contains) :+ metadataAlias, scan) + scan.output.dropRight(metadataColumns.length) :+ metadataAlias, scan) }.getOrElse(scan) val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala index 8b4b3d22920e..99445a170444 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala @@ -22,6 +22,7 @@ import java.text.SimpleDateFormat import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} @@ -289,7 +290,7 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { Seq(true, false).foreach { caseSensitive => metadataColumnsTest(s"upper/lower case when case " + s"sensitive is $caseSensitive", schemaWithNameConflicts) { (df, f0, f1) => - withSQLConf("spark.sql.caseSensitive" -> caseSensitive.toString) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { // file schema: name, age, _METADATA.id, _METADATA.university if (caseSensitive) { From 3516e4e0bca2b3e50e124c58e5993bc2349ced8f Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 15:42:17 +0800 Subject: [PATCH 16/21] missing comments --- .../spark/sql/execution/datasources/FileScanRDD.scala | 8 ++++---- .../execution/datasources/FileSourceStrategySuite.scala | 3 ++- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 7600e378186b..2ee1cb917eaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -69,7 +69,7 @@ class FileScanRDD( @transient private val sparkSession: SparkSession, readFunction: (PartitionedFile) => Iterator[InternalRow], @transient val filePartitions: Seq[FilePartition], - val requiredSchema: StructType = StructType(Seq.empty), + val readDataSchema: StructType, val metadataColumns: Seq[AttributeReference] = Seq.empty) extends RDD[InternalRow](sparkSession.sparkContext, Nil) { @@ -126,10 +126,10 @@ class FileScanRDD( // metadata columns internal row, will only be updated when the current file is changed @volatile private var metadataColumnsInternalRow: InternalRow = _ // an unsafe joiner to join an unsafe row with the metadata unsafe row - lazy private val metadataUnsafeRowJoiner = - GenerateUnsafeRowJoiner.create(requiredSchema, metadataColumns.toStructType) + private lazy val metadataUnsafeRowJoiner = + GenerateUnsafeRowJoiner.create(readDataSchema, metadataColumns.toStructType) // metadata columns unsafe row converter - lazy private val unsafeRowConverter = { + private lazy val unsafeRowConverter = { val metadataColumnsDataTypes = metadataColumns.map(_.dataType).toArray val converter = UnsafeProjection.create(metadataColumnsDataTypes) (row: InternalRow) => converter(row) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 4bfb7cf59d08..634016664dfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -292,7 +292,8 @@ class FileSourceStrategySuite extends QueryTest with SharedSparkSession with Pre val fakeRDD = new FileScanRDD( spark, (file: PartitionedFile) => Iterator.empty, - Seq(partition) + Seq(partition), + StructType(Seq.empty) ) assertResult(Set("host0", "host1", "host2")) { From 00bda9079c0210ab11f9796b64b6eaf17978eb5c Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 16:24:12 +0800 Subject: [PATCH 17/21] clean up with unsafe projection --- .../execution/datasources/FileScanRDD.scala | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 2ee1cb917eaa..e31efc2e58d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -29,7 +29,6 @@ import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow, UnsafeProjection, UnsafeRow} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeRowJoiner import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.FileFormat._ import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} @@ -121,18 +120,13 @@ class FileScanRDD( // FILE METADATA METHODS // /////////////////////////// - // metadata columns unsafe row, will only be updated when the current file is changed - @volatile private var metadataColumnsUnsafeRow: UnsafeRow = _ - // metadata columns internal row, will only be updated when the current file is changed + // a metadata columns internal row, will only be updated when the current file is changed @volatile private var metadataColumnsInternalRow: InternalRow = _ - // an unsafe joiner to join an unsafe row with the metadata unsafe row - private lazy val metadataUnsafeRowJoiner = - GenerateUnsafeRowJoiner.create(readDataSchema, metadataColumns.toStructType) - // metadata columns unsafe row converter - private lazy val unsafeRowConverter = { - val metadataColumnsDataTypes = metadataColumns.map(_.dataType).toArray - val converter = UnsafeProjection.create(metadataColumnsDataTypes) - (row: InternalRow) => converter(row) + // an unsafe projection to convert a joined internal row to an unsafe row + private lazy val projection = { + val joinedExpressions = + readDataSchema.fields.map(_.dataType) ++ metadataColumns.map(_.dataType) + UnsafeProjection.create(joinedExpressions) } /** @@ -142,7 +136,6 @@ class FileScanRDD( private def updateMetadataColumns(): Unit = { if (metadataColumns.nonEmpty) { if (currentFile == null) { - metadataColumnsUnsafeRow = null metadataColumnsInternalRow = null } else { // construct an internal row @@ -155,8 +148,6 @@ class FileScanRDD( case FILE_MODIFICATION_TIME => currentFile.modificationTime } ) - // convert the internal row to an unsafe row - metadataColumnsUnsafeRow = unsafeRowConverter(metadataColumnsInternalRow) } } } @@ -210,7 +201,7 @@ class FileScanRDD( new ColumnarBatch( Array.tabulate(c.numCols())(c.column) ++ createMetadataColumnVector(c), c.numRows()) - case u: UnsafeRow => metadataUnsafeRowJoiner.join(u, metadataColumnsUnsafeRow) + case u: UnsafeRow => projection.apply(new JoinedRow(u, metadataColumnsInternalRow)) case i: InternalRow => new JoinedRow(i, metadataColumnsInternalRow) } } else { From afa0a835f1e3f2ec30ee19ba83148d7aaec4a8f5 Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 16:46:23 +0800 Subject: [PATCH 18/21] great cleanup --- .../execution/datasources/FileScanRDD.scala | 34 ++++++++----------- 1 file changed, 15 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index e31efc2e58d8..88a389b044a0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, JoinedRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericInternalRow, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.datasources.FileFormat._ import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} @@ -120,8 +120,9 @@ class FileScanRDD( // FILE METADATA METHODS // /////////////////////////// - // a metadata columns internal row, will only be updated when the current file is changed - @volatile private var metadataColumnsInternalRow: InternalRow = _ + // a metadata internal row, will only be updated when the current file is changed + val metadataRow: InternalRow = new GenericInternalRow(metadataColumns.length) + // an unsafe projection to convert a joined internal row to an unsafe row private lazy val projection = { val joinedExpressions = @@ -134,20 +135,15 @@ class FileScanRDD( * Only update metadata columns when `currentFile` is changed. */ private def updateMetadataColumns(): Unit = { - if (metadataColumns.nonEmpty) { - if (currentFile == null) { - metadataColumnsInternalRow = null - } else { - // construct an internal row - val path = new Path(currentFile.filePath) - metadataColumnsInternalRow = InternalRow.fromSeq( - metadataColumns.map(_.name).map { - case FILE_PATH => UTF8String.fromString(path.toString) - case FILE_NAME => UTF8String.fromString(path.getName) - case FILE_SIZE => currentFile.fileSize - case FILE_MODIFICATION_TIME => currentFile.modificationTime - } - ) + if (metadataColumns.nonEmpty && currentFile != null) { + val path = new Path(currentFile.filePath) + metadataColumns.zipWithIndex.foreach { case (attr, i) => + attr.name match { + case FILE_PATH => metadataRow.update(i, UTF8String.fromString(path.toString)) + case FILE_NAME => metadataRow.update(i, UTF8String.fromString(path.getName)) + case FILE_SIZE => metadataRow.update(i, currentFile.fileSize) + case FILE_MODIFICATION_TIME => metadataRow.update(i, currentFile.modificationTime) + } } } } @@ -201,8 +197,8 @@ class FileScanRDD( new ColumnarBatch( Array.tabulate(c.numCols())(c.column) ++ createMetadataColumnVector(c), c.numRows()) - case u: UnsafeRow => projection.apply(new JoinedRow(u, metadataColumnsInternalRow)) - case i: InternalRow => new JoinedRow(i, metadataColumnsInternalRow) + case u: UnsafeRow => projection.apply(new JoinedRow(u, metadataRow)) + case i: InternalRow => new JoinedRow(i, metadataRow) } } else { nextElement From 65e79ab8ed6ce1f04fb7affeeb946f321802ce29 Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 16:53:30 +0800 Subject: [PATCH 19/21] name nit --- .../spark/sql/execution/datasources/FileScanRDD.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 88a389b044a0..0e11fb6ef094 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -132,9 +132,9 @@ class FileScanRDD( /** * For each partitioned file, metadata columns for each record in the file are exactly same. - * Only update metadata columns when `currentFile` is changed. + * Only update metadata row when `currentFile` is changed. */ - private def updateMetadataColumns(): Unit = { + private def updateMetadataRow(): Unit = { if (metadataColumns.nonEmpty && currentFile != null) { val path = new Path(currentFile.filePath) metadataColumns.zipWithIndex.foreach { case (attr, i) => @@ -236,7 +236,7 @@ class FileScanRDD( private def nextIterator(): Boolean = { if (files.hasNext) { currentFile = files.next() - updateMetadataColumns() + updateMetadataRow() logInfo(s"Reading File $currentFile") // Sets InputFileBlockHolder for the file block's information InputFileBlockHolder.set(currentFile.filePath, currentFile.start, currentFile.length) @@ -304,7 +304,7 @@ class FileScanRDD( } } else { currentFile = null - updateMetadataColumns() + updateMetadataRow() InputFileBlockHolder.unset() false } From 3b3d635870a44cf72b1acb6ee57554046988dc71 Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 18:09:21 +0800 Subject: [PATCH 20/21] change modification_time to TimestampType --- .../spark/sql/execution/PartitionedFileUtil.scala | 4 ++-- .../spark/sql/execution/datasources/FileFormat.scala | 4 ++-- .../spark/sql/execution/datasources/FileScanRDD.scala | 2 +- .../datasources/FileMetadataStructSuite.scala | 11 ++++++----- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala index 4cccd4132e91..28d379af1fae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -37,7 +37,7 @@ object PartitionedFileUtil { val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining val hosts = getBlockHosts(getBlockLocations(file), offset, size) PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts, - file.getModificationTime, file.getLen) + file.getModificationTime * 1000L, file.getLen) } } else { Seq(getPartitionedFile(file, filePath, partitionValues)) @@ -50,7 +50,7 @@ object PartitionedFileUtil { partitionValues: InternalRow): PartitionedFile = { val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts, - file.getModificationTime, file.getLen) + file.getModificationTime * 1000L, file.getLen) } private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 538e4e3f14f1..c3bcf06b6e5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjectio import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{DataType, LongType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{DataType, LongType, StringType, StructField, StructType, TimestampType} /** @@ -188,7 +188,7 @@ object FileFormat { .add(StructField(FILE_PATH, StringType)) .add(StructField(FILE_NAME, StringType)) .add(StructField(FILE_SIZE, LongType)) - .add(StructField(FILE_MODIFICATION_TIME, LongType)) + .add(StructField(FILE_MODIFICATION_TIME, TimestampType)) // create a file metadata struct col def createFileMetadataCol: AttributeReference = MetadataAttribute(METADATA_NAME, METADATA_STRUCT) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 0e11fb6ef094..f816a1a90e34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.NextIterator * @param filePath URI of the file to read * @param start the beginning offset (in bytes) of the block. * @param length number of bytes to read. - * @param modificationTime The modification time of the input file, in milliseconds. + * @param modificationTime The modification time of the input file, in microseconds. * @param fileSize The length of the input file (not the block), in bytes. */ case class PartitionedFile( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala index 99445a170444..fffac885da5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources import java.io.File +import java.sql.Timestamp import java.text.SimpleDateFormat import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} @@ -101,13 +102,13 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { METADATA_FILE_PATH -> realF0.toURI.toString, METADATA_FILE_NAME -> realF0.getName, METADATA_FILE_SIZE -> realF0.length(), - METADATA_FILE_MODIFICATION_TIME -> realF0.lastModified() + METADATA_FILE_MODIFICATION_TIME -> new Timestamp(realF0.lastModified()) ) val f1Metadata = Map( METADATA_FILE_PATH -> realF1.toURI.toString, METADATA_FILE_NAME -> realF1.getName, METADATA_FILE_SIZE -> realF1.length(), - METADATA_FILE_MODIFICATION_TIME -> realF1.lastModified() + METADATA_FILE_MODIFICATION_TIME -> new Timestamp(realF1.lastModified()) ) f(df, f0Metadata, f1Metadata) @@ -158,9 +159,9 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession { df.select( // substring of file name substring(col(METADATA_FILE_NAME), 1, 3), - // convert timestamp in millis to unixtime and to date format - from_unixtime(col(METADATA_FILE_MODIFICATION_TIME).divide(lit(1000)), "yyyy-MM") - .as("_file_modification_date"), + // format timestamp + date_format(col(METADATA_FILE_MODIFICATION_TIME), "yyyy-MM") + .as("_file_modification_year_month"), // convert to kb col(METADATA_FILE_SIZE).divide(lit(1024)).as("_file_size_kb"), // get the file format From 4400f6ac93359eb15f498f19581247dc9744d989 Mon Sep 17 00:00:00 2001 From: yaohua Date: Tue, 21 Dec 2021 18:58:33 +0800 Subject: [PATCH 21/21] 1000 in filescanrdd --- .../spark/sql/execution/PartitionedFileUtil.scala | 4 ++-- .../spark/sql/execution/datasources/FileScanRDD.scala | 11 ++++++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala index 28d379af1fae..4cccd4132e91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -37,7 +37,7 @@ object PartitionedFileUtil { val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining val hosts = getBlockHosts(getBlockLocations(file), offset, size) PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts, - file.getModificationTime * 1000L, file.getLen) + file.getModificationTime, file.getLen) } } else { Seq(getPartitionedFile(file, filePath, partitionValues)) @@ -50,7 +50,7 @@ object PartitionedFileUtil { partitionValues: InternalRow): PartitionedFile = { val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts, - file.getModificationTime * 1000L, file.getLen) + file.getModificationTime, file.getLen) } private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index f816a1a90e34..47f279babef5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.NextIterator * @param filePath URI of the file to read * @param start the beginning offset (in bytes) of the block. * @param length number of bytes to read. - * @param modificationTime The modification time of the input file, in microseconds. + * @param modificationTime The modification time of the input file, in milliseconds. * @param fileSize The length of the input file (not the block), in bytes. */ case class PartitionedFile( @@ -142,7 +142,10 @@ class FileScanRDD( case FILE_PATH => metadataRow.update(i, UTF8String.fromString(path.toString)) case FILE_NAME => metadataRow.update(i, UTF8String.fromString(path.getName)) case FILE_SIZE => metadataRow.update(i, currentFile.fileSize) - case FILE_MODIFICATION_TIME => metadataRow.update(i, currentFile.modificationTime) + case FILE_MODIFICATION_TIME => + // the modificationTime from the file is in millisecond, + // while internally, the TimestampType is stored in microsecond + metadataRow.update(i, currentFile.modificationTime * 1000L) } } } @@ -181,7 +184,9 @@ class FileScanRDD( columnVector case FILE_MODIFICATION_TIME => val columnVector = new OnHeapColumnVector(c.numRows(), LongType) - columnVector.putLongs(0, c.numRows(), currentFile.modificationTime) + // the modificationTime from the file is in millisecond, + // while internally, the TimestampType is stored in microsecond + columnVector.putLongs(0, c.numRows(), currentFile.modificationTime * 1000L) columnVector }.toArray }