This is an automated email from the ASF dual-hosted git repository.

forwardxu pushed a commit to branch release-0.12.1
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 9c94e388fecce8d02388a81f0da769d52fe5319c
Author: shaoxiong.zhan <[email protected]>
AuthorDate: Fri Sep 16 11:00:33 2022 +0800

    adapt tspark changes: backport 3.3 VectorizedParquetReader related code to 
3.1
---
 .../parquet/Spark31HoodieParquetFileFormat.scala     | 20 +++++++++++---------
 1 file changed, 11 insertions(+), 9 deletions(-)

diff --git 
a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
 
b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
index ca41490fc0..712c9c6d3e 100644
--- 
a/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
+++ 
b/hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark31HoodieParquetFileFormat.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.FileSplit
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType}
 import org.apache.hudi.HoodieSparkUtils
@@ -135,14 +136,11 @@ class Spark31HoodieParquetFileFormat(private val 
shouldAppendPartitionValues: Bo
       assert(!shouldAppendPartitionValues || 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)
+      /**
+       * from https://github.com/apache/spark/pull/29542
+       * must use org.apache.hadoop.mapred.FileSplit
+       */
+      val split = new FileSplit(filePath, file.start, file.length, 
Array.empty[String])
 
       val sharedConf = broadcastedHadoopConf.value.value
 
@@ -170,7 +168,11 @@ class Spark31HoodieParquetFileFormat(private val 
shouldAppendPartitionValues: Bo
       // Try to push down filters when filter push-down is enabled.
       val pushed = if (enableParquetFilterPushDown) {
         val parquetSchema = footerFileMetaData.getSchema
-        val parquetFilters = if (HoodieSparkUtils.gteqSpark3_1_3) {
+        /**
+         * hard code for adaption, because tspark port 3.3 api to 3.1
+         */
+        val ctor = classOf[ParquetFilters].getConstructors.head
+        val parquetFilters = if (8.equals(ctor.getParameterCount) || 
HoodieSparkUtils.gteqSpark3_1_3) {
           createParquetFilters(
             parquetSchema,
             pushDownDate,

Reply via email to