alexeykudinkin commented on code in PR #5364:
URL: https://github.com/apache/hudi/pull/5364#discussion_r853515259
##########
hudi-spark-datasource/hudi-spark3.1.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark312HoodieParquetFileFormat.scala:
##########
@@ -17,279 +17,301 @@
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.HoodieSparkUtils
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
import org.apache.hudi.common.fs.FSUtils
-import org.apache.hudi.HoodieSparkUtils
-import org.apache.hudi.common.util.InternalSchemaCache
+import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
+import org.apache.hudi.common.util.{InternalSchemaCache, StringUtils}
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.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
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.{Cast, JoinedRow}
import
org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.{Cast, JoinedRow}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import
org.apache.spark.sql.execution.datasources.parquet.Spark312HoodieParquetFileFormat.{createParquetFilters,
pruneInternalSchema, rebuildFilterFromParquet}
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))
+import java.net.URI
+
+
+/**
+ * This class is an extension of [[ParquetFileFormat]] overriding
Spark-specific behavior
+ * that's not possible to customize in any other way
+ *
+ * NOTE: This is a version of [[AvroDeserializer]] impl from Spark 3.1.2 w/ w/
the following changes applied to it:
+ * <ol>
+ * <li>Avoiding appending partition values to the rows read from the data
file</li>
+ * <li>Schema on-read</li>
+ * </ol>
+ */
+class Spark312HoodieParquetFileFormat(private val shouldAppendPartitionValues:
Boolean) extends ParquetFileFormat {
+
+ override def buildReaderWithPartitionValues(sparkSession: SparkSession,
Review Comment:
Github UI has hard time reflecting the changes properly:
1. Had to remove top-level conditional (since this FileFormat is now used to
control whether partition values will be appended)
2. Did minor cleanup for things related to handling of `InternalSchema` to
make sure those are not failing w/ NPEs
3. Adding changes to handle `shouldAppendPartitionValues`
NOTE: Copy both of those into IDEA scratchpad to be able to compare them
side by side in a more meaningful way
##########
hudi-spark-datasource/hudi-spark3/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark32HoodieParquetFileFormat.scala:
##########
@@ -27,233 +25,262 @@ 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.StringUtils.isNullOrEmpty
import org.apache.hudi.common.util.collection.Pair
import org.apache.hudi.internal.schema.InternalSchema
import org.apache.hudi.internal.schema.action.InternalSchemaMerger
import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
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.parquet.hadoop.{ParquetInputFormat, ParquetRecordReader}
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.parquet.Spark32HoodieParquetFileFormat.{pruneInternalSchema,
rebuildFilterFromParquet}
import org.apache.spark.sql.execution.datasources.{DataSourceUtils,
PartitionedFile, RecordReaderIterator}
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 Spark32HoodieParquetFileFormat 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))
+import java.net.URI
+
+/**
+ * This class is an extension of [[ParquetFileFormat]] overriding
Spark-specific behavior
+ * that's not possible to customize in any other way
+ *
+ * NOTE: This is a version of [[AvroDeserializer]] impl from Spark 3.2.1 w/ w/
the following changes applied to it:
+ * <ol>
+ * <li>Avoiding appending partition values to the rows read from the data
file</li>
+ * <li>Schema on-read</li>
+ * </ol>
+ */
+class Spark32HoodieParquetFileFormat(private val shouldAppendPartitionValues:
Boolean) extends ParquetFileFormat {
Review Comment:
Same comments as for Spark 3.1
--
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]