xiarixiaoyao commented on a change in pull request #4910:
URL: https://github.com/apache/hudi/pull/4910#discussion_r840565245



##########
File path: 
hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark312HoodieParquetFileFormat.scala
##########
@@ -0,0 +1,365 @@
+/*
+ * 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 java.net.URI
+import java.util
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
+import org.apache.hudi.client.utils.SparkInternalSchemaConverter
+import org.apache.hudi.common.fs.FSUtils
+import org.apache.hudi.common.util.InternalSchemaCache
+import org.apache.hudi.common.util.collection.Pair
+import org.apache.hudi.internal.schema.InternalSchema
+import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
+import org.apache.hudi.internal.schema.action.InternalSchemaMerger
+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, 
ParquetRecordReader}
+
+import org.apache.spark.SPARK_VERSION
+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.{Cast, JoinedRow}
+import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.{DataSourceUtils, 
PartitionedFile, RecordReaderIterator}
+import org.apache.spark.sql.execution.datasources.parquet._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.{AtomicType, DataType, StructField, 
StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+class Spark312HoodieParquetFileFormat extends ParquetFileFormat {
+
+  // reference ParquetFileFormat from spark project
+  override def buildReaderWithPartitionValues(
+                                               sparkSession: SparkSession,
+                                               dataSchema: StructType,
+                                               partitionSchema: StructType,
+                                               requiredSchema: StructType,
+                                               filters: Seq[Filter],
+                                               options: Map[String, String],
+                                               hadoopConf: Configuration): 
PartitionedFile => Iterator[InternalRow] = {
+    if (hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, 
"").isEmpty) {
+      // fallback to origin parquet File read
+      super.buildReaderWithPartitionValues(sparkSession, dataSchema, 
partitionSchema, requiredSchema, filters, options, hadoopConf)
+    } else {
+      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.NESTED_SCHEMA_PRUNING_ENABLED.key,
+        sparkSession.sessionState.conf.nestedSchemaPruningEnabled)
+      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)
+      // for dataSource v1, we have no method to do project for spark physical 
plan.
+      // it's safe to do cols project here.
+      val internalSchemaString = 
hadoopConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
+      val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
+      if (querySchemaOption.isPresent && !requiredSchema.isEmpty) {
+        val prunedSchema = 
SparkInternalSchemaConverter.convertAndPruneStructTypeToInternalSchema(requiredSchema,
 querySchemaOption.get())
+        hadoopConf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, 
SerDeHelper.toJson(prunedSchema))
+      }
+      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 enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled
+      val enableVectorizedReader: Boolean =
+        sqlConf.parquetVectorizedReaderEnabled &&
+          resultSchema.forall(_.dataType.isInstanceOf[AtomicType])
+      val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled
+      val timestampConversion: Boolean = 
sqlConf.isParquetINT96TimestampConversion
+      val capacity = sqlConf.parquetVectorizedReaderBatchSize
+      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 filePath = new Path(new URI(file.filePath))
+        val split =
+          new org.apache.parquet.hadoop.ParquetInputSplit(
+            filePath,
+            file.start,
+            file.start + file.length,
+            file.length,
+            Array.empty,
+            null)
+        val sharedConf = broadcastedHadoopConf.value.value
+        // do deal with internalSchema
+        val internalSchemaString = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA)
+        // querySchema must be a pruned schema.
+        val querySchemaOption = SerDeHelper.fromJson(internalSchemaString)
+        val internalSchemaChangeEnabled = if (internalSchemaString.isEmpty || 
!querySchemaOption.isPresent) false else true
+        val tablePath = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
+        val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
+        val fileSchema = if (internalSchemaChangeEnabled) {
+          val validCommits = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
+          InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, 
tablePath, sharedConf, if (validCommits == null) "" else validCommits)
+        } else {
+          // this should not happened, searchSchemaAndCache will deal with 
correctly.
+          null
+        }
+
+        lazy val footerFileMetaData =
+          ParquetFileReader.readFooter(sharedConf, filePath, 
SKIP_ROW_GROUPS).getFileMetaData
+        val datetimeRebaseMode = DataSourceUtils.datetimeRebaseMode(
+          footerFileMetaData.getKeyValueMetaData.get,
+          SQLConf.get.getConf(SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_READ))
+        // Try to push down filters when filter push-down is enabled.
+        val pushed = if (enableParquetFilterPushDown) {
+          val parquetSchema = footerFileMetaData.getSchema
+          val parquetFilters = if (SPARK_VERSION.startsWith("3.1.3")) {

Review comment:
       yes, you are right,   maybe spark3.1.4 will be released, although this 
possibility is very low
   now we already have a ```SparkVersion class that parses SparkVersion```  let 
me fixed this
   already fixed




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to