You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/12/22 07:48:01 UTC

[GitHub] [spark] c21 commented on a change in pull request #34575: [SPARK-37273][SQL] Support hidden file metadata columns in Spark SQL

c21 commented on a change in pull request #34575:
URL: https://github.com/apache/spark/pull/34575#discussion_r773663087



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
##########
@@ -103,6 +115,101 @@ class FileScanRDD(
         context.killTaskIfInterrupted()
         (currentIterator != null && currentIterator.hasNext) || nextIterator()
       }
+
+      ///////////////////////////
+      // FILE METADATA METHODS //
+      ///////////////////////////
+
+      // 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 =
+          readDataSchema.fields.map(_.dataType) ++ metadataColumns.map(_.dataType)
+        UnsafeProjection.create(joinedExpressions)
+      }
+
+      /**
+       * For each partitioned file, metadata columns for each record in the file are exactly same.
+       * Only update metadata row when `currentFile` is changed.
+       */
+      private def updateMetadataRow(): Unit = {
+        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 =>
+                // the modificationTime from the file is in millisecond,
+                // while internally, the TimestampType is stored in microsecond
+                metadataRow.update(i, currentFile.modificationTime * 1000L)
+            }
+          }
+        }
+      }
+
+      /**
+       * Create a writable column vector containing all required metadata columns
+       */
+      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
+        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

Review comment:
       It looks like for each batch of input rows, we need to recreate new column vector onheap, and write the same constant values per each row (i.e. file path, file name, file size, etc). Just wondering the performance penalty when reading a large table, how big of table have we tested?
   
   Maybe a simple optimization here is to come up with something like `ConstantColumnVector`, where for each row, all values are same, and we only need to save one copy of value.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
##########
@@ -171,6 +171,29 @@ trait FileFormat {
   def supportFieldName(name: String): Boolean = true
 }
 
+object FileFormat {
+
+  val FILE_PATH = "file_path"
+
+  val FILE_NAME = "file_name"

Review comment:
       wondering do we also plan to deprecate existing expression `InputFileName` in Spark?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala
##########
@@ -212,7 +213,19 @@ object FileSourceStrategy extends Strategy with PredicateHelper with Logging {
       val outputSchema = readDataColumns.toStructType
       logInfo(s"Output Data Schema: ${outputSchema.simpleString(5)}")
 
-      val outputAttributes = readDataColumns ++ partitionColumns
+      val metadataStructOpt = requiredAttributes.collectFirst {
+        case MetadataAttribute(attr) => attr
+      }
+
+      // TODO (yaohua): should be able to prune the metadata struct only containing what needed

Review comment:
       nit: Shall we file a JIRA?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
##########
@@ -103,6 +115,101 @@ class FileScanRDD(
         context.killTaskIfInterrupted()
         (currentIterator != null && currentIterator.hasNext) || nextIterator()
       }
+
+      ///////////////////////////
+      // FILE METADATA METHODS //
+      ///////////////////////////
+
+      // 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 =
+          readDataSchema.fields.map(_.dataType) ++ metadataColumns.map(_.dataType)
+        UnsafeProjection.create(joinedExpressions)
+      }
+
+      /**
+       * For each partitioned file, metadata columns for each record in the file are exactly same.
+       * Only update metadata row when `currentFile` is changed.
+       */
+      private def updateMetadataRow(): Unit = {
+        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 =>
+                // the modificationTime from the file is in millisecond,
+                // while internally, the TimestampType is stored in microsecond
+                metadataRow.update(i, currentFile.modificationTime * 1000L)
+            }
+          }
+        }
+      }
+
+      /**
+       * Create a writable column vector containing all required metadata columns
+       */
+      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
+        metadataColumns.map(_.name).map {

Review comment:
       We should already know how to fill column vector for each metadata column, so the pattern matching can be done outside of execution, and here it does not need to do pattern matching per batch.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org