You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/06/10 16:36:03 UTC

[GitHub] [hudi] garyli1019 commented on a change in pull request #1722: [HUDI-69] Support Spark Datasource for MOR table

garyli1019 commented on a change in pull request #1722:
URL: https://github.com/apache/hudi/pull/1722#discussion_r438261014



##########
File path: hudi-spark/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieRealtimeInputFormat.scala
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.parquet
+
+import org.apache.hudi.hadoop.realtime.HoodieRealtimeFileSplit
+import org.apache.hudi.realtime.HoodieRealtimeParquetRecordReader
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.{FileSplit, JobConf}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.parquet.filter2.compat.FilterCompat
+import org.apache.parquet.filter2.predicate.FilterApi
+import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS
+import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+import scala.collection.JavaConverters._
+
+/**
+ * This class is an extension of ParquetFileFormat from Spark SQL.
+ * The file split, record reader, record reader iterator are customized to read Hudi MOR table.
+ */
+class HoodieRealtimeInputFormat extends ParquetFileFormat {
+  //TODO: Better usage of this short name.
+  override def shortName(): String = "hudi.realtime"
+  override def toString(): String = "hudi.realtime"
+
+  override def buildReaderWithPartitionValues(sparkSession: SparkSession,
+                                               dataSchema: StructType,
+                                               partitionSchema: StructType,
+                                               requiredSchema: StructType,
+                                               filters: Seq[Filter],
+                                               options: Map[String, String],
+                                               hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = {
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName)
+    hadoopConf.set(
+      ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      ParquetWriteSupport.SPARK_ROW_SCHEMA,
+      requiredSchema.json)
+    hadoopConf.set(
+      SQLConf.SESSION_LOCAL_TIMEZONE.key,
+      sparkSession.sessionState.conf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(
+      SQLConf.CASE_SENSITIVE.key,
+      sparkSession.sessionState.conf.caseSensitiveAnalysis)
+
+    ParquetWriteSupport.setSchema(requiredSchema, hadoopConf)
+
+    // Sets flags for `ParquetToSparkSchemaConverter`
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_BINARY_AS_STRING.key,
+      sparkSession.sessionState.conf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(
+      SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
+      sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+
+    val broadcastedHadoopConf =
+      sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
+
+    // TODO: if you move this into the closure it reverts to the default values.
+    // If true, enable using the custom RecordReader for parquet. This only works for
+    // a subset of the types (no complex types).
+    val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields)
+    val sqlConf = sparkSession.sessionState.conf
+    val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
+    val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion
+    val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown
+    // Whole stage codegen (PhysicalRDD) is able to deal with batches directly
+    val returningBatch = supportBatch(sparkSession, resultSchema)
+    val pushDownDate = sqlConf.parquetFilterPushDownDate
+    val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp
+    val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal
+    val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith
+    val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold
+    val isCaseSensitive = sqlConf.caseSensitiveAnalysis
+
+    (file: PartitionedFile) => {
+      assert(file.partitionValues.numFields == partitionSchema.size)
+
+      val sharedConf = broadcastedHadoopConf.value.value
+      val fileSplit =
+        new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, new Array[String](0))
+      val filePath = fileSplit.getPath
+
+      val basePath = sharedConf.get("mapreduce.input.fileinputformat.inputdir")
+      val maxCommitTime = sharedConf.get("hoodie.realtime.last.commit")
+      // Read the log file path from the option
+      val logPathStr = options.getOrElse(fileSplit.getPath.toString, "").split(",")
+      log.debug(s"fileSplit.getPath in HoodieRealtimeInputFormat: ${fileSplit.getPath} and ${fileSplit.getPath.getName}")
+      log.debug(s"logPath in HoodieRealtimeInputFormat: ${logPathStr.toString}")
+      val hoodieRealtimeFileSplit = new HoodieRealtimeFileSplit(fileSplit, basePath, logPathStr.toList.asJava, maxCommitTime)
+
+      lazy val footerFileMetaData =
+        ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      // Try to push down filters when filter push-down is enabled.
+      val pushed = if (enableParquetFilterPushDown) {
+        val parquetSchema = footerFileMetaData.getSchema
+        val parquetFilters = new ParquetFilters(pushDownDate, pushDownTimestamp, pushDownDecimal,
+          pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive)
+        filters
+          // Collects all converted Parquet filter predicates. Notice that not all predicates can be
+          // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap`
+          // is used here.
+          .flatMap(parquetFilters.createFilter(parquetSchema, _))
+          .reduceOption(FilterApi.and)
+      } else {
+        None
+      }
+
+      // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps'
+      // *only* if the file was created by something other than "parquet-mr", so check the actual
+      // writer here for this file.  We have to do this per-file, as each file in the table may
+      // have different writers.
+      // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads.
+      def isCreatedByParquetMr: Boolean =
+        footerFileMetaData.getCreatedBy().startsWith("parquet-mr")
+
+      val convertTz =
+        if (timestampConversion && !isCreatedByParquetMr) {
+          Some(DateTimeUtils.getTimeZone(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)))
+        } else {
+          None
+        }
+
+      val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
+      val hadoopAttemptContext =
+        new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId)
+
+      // Try to push down filters when filter push-down is enabled.
+      // Notice: This push-down is RowGroups level, not individual records.
+      if (pushed.isDefined) {
+        ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get)
+      }
+      val taskContext = Option(TaskContext.get())
+      //TODO: Support the vectorized reader.
+      logDebug(s"Falling back to parquet-mr")
+      // ParquetRecordReader returns UnsafeRow
+      val reader = if (pushed.isDefined && enableRecordFilter) {
+        val parquetFilter = FilterCompat.get(pushed.get, null)
+        new HoodieRealtimeParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter, hoodieRealtimeFileSplit, new JobConf(sharedConf))
+      } else {
+        new HoodieRealtimeParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), hoodieRealtimeFileSplit, new JobConf(sharedConf))
+      }
+      val iter = new HoodieParquetRecordReaderIterator(reader)
+      // SPARK-23457 Register a task completion lister before `initialization`.
+      taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close()))
+      reader.initialize(hoodieRealtimeFileSplit, hadoopAttemptContext)
+      iter.init()
+
+      val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
+      val joinedRow = new JoinedRow()
+      val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+
+      // This is a horrible erasure hack...  if we type the iterator above, then it actually check

Review comment:
       yes this is from Spark 2.4 https://github.com/apache/spark/blob/branch-2.4/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L445
   This horrible erasure hack was updated in the latest master though...




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

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