yihua commented on code in PR #10954:
URL: https://github.com/apache/hudi/pull/10954#discussion_r1560196572


##########
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.lib.input.FileSplit
+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, 
ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.avro.AvroDeserializer
+import org.apache.spark.sql.catalyst.InternalRow
+import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.{PartitionedFile, 
RecordReaderIterator}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AtomicType, StructField, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+
+object Spark24HoodieParquetReader {
+
+  /**
+   * Get properties needed to read a parquet file
+   *
+   * @param vectorized true if vectorized reading is not prohibited due to 
schema, reading mode, etc
+   * @param sqlConf    the [[SQLConf]] used for the read
+   * @param options    passed as a param to the file format
+   * @param hadoopConf some configs will be set for the hadoopConf
+   * @return map of properties needed for reading a parquet file
+   */
+  def getPropsForReadingParquet(vectorized: Boolean,
+                                sqlConf: SQLConf,
+                                options: Map[String, String],
+                                hadoopConf: Configuration): Map[String, 
String] = {
+    //set hadoopconf
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, 
classOf[ParquetReadSupport].getName)
+    hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, 
sqlConf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key, 
sqlConf.caseSensitiveAnalysis)
+    hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, 
sqlConf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, 
sqlConf.isParquetINT96AsTimestamp)
+
+    Map(
+      "enableVectorizedReader" -> vectorized.toString,
+      "enableParquetFilterPushDown" -> sqlConf.parquetFilterPushDown.toString,
+      "pushDownDate" -> sqlConf.parquetFilterPushDownDate.toString,
+      "pushDownTimestamp" -> sqlConf.parquetFilterPushDownTimestamp.toString,
+      "pushDownDecimal" -> sqlConf.parquetFilterPushDownDecimal.toString,
+      "pushDownInFilterThreshold" -> 
sqlConf.parquetFilterPushDownInFilterThreshold.toString,
+      "pushDownStringStartWith" -> 
sqlConf.parquetFilterPushDownStringStartWith.toString,
+      "isCaseSensitive" -> sqlConf.caseSensitiveAnalysis.toString,
+      "timestampConversion" -> 
sqlConf.isParquetINT96TimestampConversion.toString,
+      "enableOffHeapColumnVector" -> 
sqlConf.offHeapColumnVectorEnabled.toString,
+      "capacity" -> sqlConf.parquetVectorizedReaderBatchSize.toString,
+      "returningBatch" -> sqlConf.parquetVectorizedReaderEnabled.toString,
+      "enableRecordFilter" -> sqlConf.parquetRecordFilterEnabled.toString,
+      "timeZoneId" -> sqlConf.sessionLocalTimeZone
+    )
+  }
+
+  /**
+   * Read an individual parquet file
+   * Code from ParquetFileFormat#buildReaderWithPartitionValues from Spark 
v2.4.8 adapted here
+   *
+   * @param file            parquet file to read
+   * @param requiredSchema  desired output schema of the data
+   * @param partitionSchema schema of the partition columns. Partition values 
will be appended to the end of every row
+   * @param filters         filters for data skipping. Not guaranteed to be 
used; the spark plan will also apply the filters.
+   * @param sharedConf      the hadoop conf
+   * @param extraProps      properties generated by 
[[getPropsForReadingParquet]] that are needed for reading
+   * @return iterator of rows read from the file output type says 
[[InternalRow]] but could be [[ColumnarBatch]]
+   */
+  def readParquetFile(file: PartitionedFile,

Review Comment:
   Let's still add simple test for each Spark version to make sure the new 
Parquet reader can read a parquet file.
   
   
   
   



##########
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark24HoodieParquetReader.scala:
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.lib.input.FileSplit
+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, 
ParquetRecordReader}
+import org.apache.spark.TaskContext
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.avro.AvroDeserializer
+import org.apache.spark.sql.catalyst.InternalRow
+import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow, UnsafeRow}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.datasources.{PartitionedFile, 
RecordReaderIterator}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AtomicType, StructField, StructType}
+import org.apache.spark.util.SerializableConfiguration
+
+import java.net.URI
+
+object Spark24HoodieParquetReader {
+
+  /**
+   * Get properties needed to read a parquet file
+   *
+   * @param vectorized true if vectorized reading is not prohibited due to 
schema, reading mode, etc
+   * @param sqlConf    the [[SQLConf]] used for the read
+   * @param options    passed as a param to the file format
+   * @param hadoopConf some configs will be set for the hadoopConf
+   * @return map of properties needed for reading a parquet file
+   */
+  def getPropsForReadingParquet(vectorized: Boolean,
+                                sqlConf: SQLConf,
+                                options: Map[String, String],
+                                hadoopConf: Configuration): Map[String, 
String] = {
+    //set hadoopconf
+    hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, 
classOf[ParquetReadSupport].getName)
+    hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, 
sqlConf.sessionLocalTimeZone)
+    hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key, 
sqlConf.caseSensitiveAnalysis)
+    hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, 
sqlConf.isParquetBinaryAsString)
+    hadoopConf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, 
sqlConf.isParquetINT96AsTimestamp)
+
+    Map(
+      "enableVectorizedReader" -> vectorized.toString,
+      "enableParquetFilterPushDown" -> sqlConf.parquetFilterPushDown.toString,
+      "pushDownDate" -> sqlConf.parquetFilterPushDownDate.toString,
+      "pushDownTimestamp" -> sqlConf.parquetFilterPushDownTimestamp.toString,
+      "pushDownDecimal" -> sqlConf.parquetFilterPushDownDecimal.toString,
+      "pushDownInFilterThreshold" -> 
sqlConf.parquetFilterPushDownInFilterThreshold.toString,
+      "pushDownStringStartWith" -> 
sqlConf.parquetFilterPushDownStringStartWith.toString,
+      "isCaseSensitive" -> sqlConf.caseSensitiveAnalysis.toString,
+      "timestampConversion" -> 
sqlConf.isParquetINT96TimestampConversion.toString,
+      "enableOffHeapColumnVector" -> 
sqlConf.offHeapColumnVectorEnabled.toString,
+      "capacity" -> sqlConf.parquetVectorizedReaderBatchSize.toString,
+      "returningBatch" -> sqlConf.parquetVectorizedReaderEnabled.toString,
+      "enableRecordFilter" -> sqlConf.parquetRecordFilterEnabled.toString,
+      "timeZoneId" -> sqlConf.sessionLocalTimeZone
+    )
+  }
+
+  /**
+   * Read an individual parquet file
+   * Code from ParquetFileFormat#buildReaderWithPartitionValues from Spark 
v2.4.8 adapted here
+   *
+   * @param file            parquet file to read
+   * @param requiredSchema  desired output schema of the data
+   * @param partitionSchema schema of the partition columns. Partition values 
will be appended to the end of every row
+   * @param filters         filters for data skipping. Not guaranteed to be 
used; the spark plan will also apply the filters.
+   * @param sharedConf      the hadoop conf
+   * @param extraProps      properties generated by 
[[getPropsForReadingParquet]] that are needed for reading
+   * @return iterator of rows read from the file output type says 
[[InternalRow]] but could be [[ColumnarBatch]]
+   */
+  def readParquetFile(file: PartitionedFile,

Review Comment:
   Let's still add simple test for each Spark version to make sure the new 
Parquet reader can read a parquet file.
   
   
   
   



-- 
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